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
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.
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.
- Cycle through a list of "bootstrap" Kafka URLs until we find one we can connect to. Fetch cluster metadata.
- Process fetch or produce requests, directing them to the appropriate broker based on the topic/partitions they send to or fetch from.
- If we get an appropriate error, refresh the metadata and try again.
As mentioned above the assignment of messages to partitions is something the producing client controls. That said, how should this functionality be exposed to the end-user?
Partitioning really serves two purposes in Kafka:
- It balances data and request load over brokers
- It serves as a way to divvy up processing among consumer processes while allowing local state and preserving order within the partition. We call this semantic partitioning.
For a given use case you may care about only one of these or both.
To accomplish simple load balancing a simple approach would be for the client to just round robin requests over all brokers. Another alternative, in an environment where there are many more producers than brokers, would be to have each client chose a single partition at random and publish to that. This later strategy will result in far fewer TCP connections.
Semantic partitioning means using some key in the message to assign messages to partitions. For example if you were processing a click message stream you might want to partition the stream by the user id so that all data for a particular user would go to a single consumer. To accomplish this the client can take a key associated with the message and use some hash of this key to choose the partition to which to deliver the message.
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.
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 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.
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.
- 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.
- 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.
- 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.
- Deprecation of a protocol version is done by marking an API version as deprecated in the protocol documentation.
- 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.
The following sequence is used for SASL authentication:
- Kafka
ApiVersionsRequest
may be sent by the client to obtain the version ranges of requests supported by the broker. This is optional.
- 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.
- 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.
- 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 is built out of the following primitive types.
Type |
Description |
BOOLEAN | Represents 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. |
INT8 | Represents an integer between -27 and 27-1 inclusive. |
INT16 | Represents an integer between -215 and 215-1 inclusive. The values are encoded using two bytes in network byte order (big-endian). |
INT32 | Represents an integer between -231 and 231-1 inclusive. The values are encoded using four bytes in network byte order (big-endian). |
INT64 | Represents an integer between -263 and 263-1 inclusive. The values are encoded using eight bytes in network byte order (big-endian). |
UINT32 | Represents an integer between 0 and 232-1 inclusive. The values are encoded using four bytes in network byte order (big-endian). |
VARINT | Represents an integer between -231 and 231-1 inclusive. Encoding follows the variable-length zig-zag encoding from Google Protocol Buffers. |
VARLONG | Represents an integer between -263 and 263-1 inclusive. Encoding follows the variable-length zig-zag encoding from Google Protocol Buffers. |
UUID | Represents a type 4 immutable universally unique identifier (Uuid). The values are encoded using sixteen bytes in network byte order (big-endian). |
FLOAT64 | Represents a double-precision 64-bit format IEEE 754 value. The values are encoded using eight bytes in network byte order (big-endian). |
STRING | Represents 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_STRING | Represents 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_STRING | Represents 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_STRING | Represents 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. |
BYTES | Represents a raw sequence of bytes. First the length N is given as an INT32. Then N bytes follow. |
COMPACT_BYTES | Represents a raw sequence of bytes. First the length N+1 is given as an UNSIGNED_VARINT.Then N bytes follow. |
NULLABLE_BYTES | Represents 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_BYTES | Represents 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. |
RECORDS | Represents a sequence of Kafka records as NULLABLE_BYTES. For a detailed description of records see Message Sets. |
ARRAY | Represents 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_ARRAY | Represents 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]. |
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.
All requests and responses originate from the following grammar which will be incrementally describe through the rest of this document:
RequestOrResponse => Size (RequestMessage | ResponseMessage)
Size => int32
Field | Description |
message_size | The message_size field gives the size of the subsequent request or response message in bytes. The client can read requests by first reading this 4 byte size as an integer N, and then reading and parsing the subsequent N bytes of the request. |
A description of the record batch format can be found here.
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 | -1 | False | The server experienced an unexpected error when processing the request. |
NONE | 0 | False | |
OFFSET_OUT_OF_RANGE | 1 | False | The requested offset is not within the range of offsets maintained by the server. |
CORRUPT_MESSAGE | 2 | True | This message has failed its CRC checksum, exceeds the valid size, has a null key for a compacted topic, or is otherwise corrupt. |
UNKNOWN_TOPIC_OR_PARTITION | 3 | True | This server does not host this topic-partition. |
INVALID_FETCH_SIZE | 4 | False | The requested fetch size is invalid. |
LEADER_NOT_AVAILABLE | 5 | True | There is no leader for this topic-partition as we are in the middle of a leadership election. |
NOT_LEADER_OR_FOLLOWER | 6 | True | For 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_OUT | 7 | True | The request timed out. |
BROKER_NOT_AVAILABLE | 8 | False | The broker is not available. |
REPLICA_NOT_AVAILABLE | 9 | True | The 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_LARGE | 10 | False | The request included a message larger than the max message size the server will accept. |
STALE_CONTROLLER_EPOCH | 11 | False | The controller moved to another broker. |
OFFSET_METADATA_TOO_LARGE | 12 | False | The metadata field of the offset request was too large. |
NETWORK_EXCEPTION | 13 | True | The server disconnected before a response was received. |
COORDINATOR_LOAD_IN_PROGRESS | 14 | True | The coordinator is loading and hence can't process requests. |
COORDINATOR_NOT_AVAILABLE | 15 | True | The coordinator is not available. |
NOT_COORDINATOR | 16 | True | This is not the correct coordinator. |
INVALID_TOPIC_EXCEPTION | 17 | False | The request attempted to perform an operation on an invalid topic. |
RECORD_LIST_TOO_LARGE | 18 | False | The request included message batch larger than the configured segment size on the server. |
NOT_ENOUGH_REPLICAS | 19 | True | Messages are rejected since there are fewer in-sync replicas than required. |
NOT_ENOUGH_REPLICAS_AFTER_APPEND | 20 | True | Messages are written to the log, but to fewer in-sync replicas than required. |
INVALID_REQUIRED_ACKS | 21 | False | Produce request specified an invalid value for required acks. |
ILLEGAL_GENERATION | 22 | False | Specified group generation id is not valid. |
INCONSISTENT_GROUP_PROTOCOL | 23 | False | The group member's supported protocols are incompatible with those of existing members or first group member tried to join with empty protocol type or empty protocol list. |
INVALID_GROUP_ID | 24 | False | The configured groupId is invalid. |
UNKNOWN_MEMBER_ID | 25 | False | The coordinator is not aware of this member. |
INVALID_SESSION_TIMEOUT | 26 | False | The session timeout is not within the range allowed by the broker (as configured by group.min.session.timeout.ms and group.max.session.timeout.ms). |
REBALANCE_IN_PROGRESS | 27 | False | The group is rebalancing, so a rejoin is needed. |
INVALID_COMMIT_OFFSET_SIZE | 28 | False | The committing offset data size is not valid. |
TOPIC_AUTHORIZATION_FAILED | 29 | False | Topic authorization failed. |
GROUP_AUTHORIZATION_FAILED | 30 | False | Group authorization failed. |
CLUSTER_AUTHORIZATION_FAILED | 31 | False | Cluster authorization failed. |
INVALID_TIMESTAMP | 32 | False | The timestamp of the message is out of acceptable range. |
UNSUPPORTED_SASL_MECHANISM | 33 | False | The broker does not support the requested SASL mechanism. |
ILLEGAL_SASL_STATE | 34 | False | Request is not valid given the current SASL state. |
UNSUPPORTED_VERSION | 35 | False | The version of API is not supported. |
TOPIC_ALREADY_EXISTS | 36 | False | Topic with this name already exists. |
INVALID_PARTITIONS | 37 | False | Number of partitions is below 1. |
INVALID_REPLICATION_FACTOR | 38 | False | Replication factor is below 1 or larger than the number of available brokers. |
INVALID_REPLICA_ASSIGNMENT | 39 | False | Replica assignment is invalid. |
INVALID_CONFIG | 40 | False | Configuration is invalid. |
NOT_CONTROLLER | 41 | True | This is not the correct controller for this cluster. |
INVALID_REQUEST | 42 | False | This most likely occurs because of a request being malformed by the client library or the message was sent to an incompatible broker. See the broker logs for more details. |
UNSUPPORTED_FOR_MESSAGE_FORMAT | 43 | False | The message format version on the broker does not support the request. |
POLICY_VIOLATION | 44 | False | Request parameters do not satisfy the configured policy. |
OUT_OF_ORDER_SEQUENCE_NUMBER | 45 | False | The broker received an out of order sequence number. |
DUPLICATE_SEQUENCE_NUMBER | 46 | False | The broker received a duplicate sequence number. |
INVALID_PRODUCER_EPOCH | 47 | False | Producer attempted to produce with an old epoch. |
INVALID_TXN_STATE | 48 | False | The producer attempted a transactional operation in an invalid state. |
INVALID_PRODUCER_ID_MAPPING | 49 | False | The producer attempted to use a producer id which is not currently assigned to its transactional id. |
INVALID_TRANSACTION_TIMEOUT | 50 | False | The transaction timeout is larger than the maximum value allowed by the broker (as configured by transaction.max.timeout.ms). |
CONCURRENT_TRANSACTIONS | 51 | True | The producer attempted to update a transaction while another concurrent operation on the same transaction was ongoing. |
TRANSACTION_COORDINATOR_FENCED | 52 | False | Indicates that the transaction coordinator sending a WriteTxnMarker is no longer the current coordinator for a given producer. |
TRANSACTIONAL_ID_AUTHORIZATION_FAILED | 53 | False | Transactional Id authorization failed. |
SECURITY_DISABLED | 54 | False | Security features are disabled. |
OPERATION_NOT_ATTEMPTED | 55 | False | The broker did not attempt to execute this operation. This may happen for batched RPCs where some operations in the batch failed, causing the broker to respond without trying the rest. |
KAFKA_STORAGE_ERROR | 56 | True | Disk error when trying to access log file on the disk. |
LOG_DIR_NOT_FOUND | 57 | False | The user-specified log directory is not found in the broker config. |
SASL_AUTHENTICATION_FAILED | 58 | False | SASL Authentication failed. |
UNKNOWN_PRODUCER_ID | 59 | False | This 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_PROGRESS | 60 | False | A partition reassignment is in progress. |
DELEGATION_TOKEN_AUTH_DISABLED | 61 | False | Delegation Token feature is not enabled. |
DELEGATION_TOKEN_NOT_FOUND | 62 | False | Delegation Token is not found on server. |
DELEGATION_TOKEN_OWNER_MISMATCH | 63 | False | Specified Principal is not valid Owner/Renewer. |
DELEGATION_TOKEN_REQUEST_NOT_ALLOWED | 64 | False | Delegation Token requests are not allowed on PLAINTEXT/1-way SSL channels and on delegation token authenticated channels. |
DELEGATION_TOKEN_AUTHORIZATION_FAILED | 65 | False | Delegation Token authorization failed. |
DELEGATION_TOKEN_EXPIRED | 66 | False | Delegation Token is expired. |
INVALID_PRINCIPAL_TYPE | 67 | False | Supplied principalType is not supported. |
NON_EMPTY_GROUP | 68 | False | The group is not empty. |
GROUP_ID_NOT_FOUND | 69 | False | The group id does not exist. |
FETCH_SESSION_ID_NOT_FOUND | 70 | True | The fetch session ID was not found. |
INVALID_FETCH_SESSION_EPOCH | 71 | True | The fetch session epoch is invalid. |
LISTENER_NOT_FOUND | 72 | True | There is no listener on the leader broker that matches the listener on which metadata request was processed. |
TOPIC_DELETION_DISABLED | 73 | False | Topic deletion is disabled. |
FENCED_LEADER_EPOCH | 74 | True | The leader epoch in the request is older than the epoch on the broker. |
UNKNOWN_LEADER_EPOCH | 75 | True | The leader epoch in the request is newer than the epoch on the broker. |
UNSUPPORTED_COMPRESSION_TYPE | 76 | False | The requesting client does not support the compression type of given partition. |
STALE_BROKER_EPOCH | 77 | False | Broker epoch has changed. |
OFFSET_NOT_AVAILABLE | 78 | True | The leader high watermark has not caught up from a recent leader election so the offsets cannot be guaranteed to be monotonically increasing. |
MEMBER_ID_REQUIRED | 79 | False | The group member needs to have a valid member id before actually entering a consumer group. |
PREFERRED_LEADER_NOT_AVAILABLE | 80 | True | The preferred leader was not available. |
GROUP_MAX_SIZE_REACHED | 81 | False | The consumer group has reached its max size. |
FENCED_INSTANCE_ID | 82 | False | The broker rejected this static consumer since another consumer with the same group.instance.id has registered with a different member.id. |
ELIGIBLE_LEADERS_NOT_AVAILABLE | 83 | True | Eligible topic partition leaders are not available. |
ELECTION_NOT_NEEDED | 84 | True | Leader election not needed for topic partition. |
NO_REASSIGNMENT_IN_PROGRESS | 85 | False | No partition reassignment is in progress. |
GROUP_SUBSCRIBED_TO_TOPIC | 86 | False | Deleting offsets of a topic is forbidden while the consumer group is actively subscribed to it. |
INVALID_RECORD | 87 | False | This record has failed the validation on broker and hence will be rejected. |
UNSTABLE_OFFSET_COMMIT | 88 | True | There are unstable offsets that need to be cleared. |
THROTTLING_QUOTA_EXCEEDED | 89 | True | The throttling quota has been exceeded. |
PRODUCER_FENCED | 90 | False | There is a newer producer with the same transactionalId which fences the current one. |
RESOURCE_NOT_FOUND | 91 | False | A request illegally referred to a resource that does not exist. |
DUPLICATE_RESOURCE | 92 | False | A request illegally referred to the same resource twice. |
UNACCEPTABLE_CREDENTIAL | 93 | False | Requested credential would not meet criteria for acceptability. |
INCONSISTENT_VOTER_SET | 94 | False | Indicates that the either the sender or recipient of a voter-only request is not one of the expected voters |
INVALID_UPDATE_VERSION | 95 | False | The given update version was invalid. |
FEATURE_UPDATE_FAILED | 96 | False | Unable to update finalized features due to an unexpected server error. |
PRINCIPAL_DESERIALIZATION_FAILURE | 97 | False | Request principal deserialization failed during forwarding. This indicates an internal error on the broker cluster security setup. |
SNAPSHOT_NOT_FOUND | 98 | False | Requested snapshot was not found |
POSITION_OUT_OF_RANGE | 99 | False | Requested position is not greater than or equal to zero, and less than the size of the snapshot. |
UNKNOWN_TOPIC_ID | 100 | True | This server does not host this topic ID. |
DUPLICATE_BROKER_REGISTRATION | 101 | False | This broker ID is already in use. |
BROKER_ID_NOT_REGISTERED | 102 | False | The given broker ID was not registered. |
INCONSISTENT_TOPIC_ID | 103 | True | The log's topic ID did not match the topic ID in the request |
INCONSISTENT_CLUSTER_ID | 104 | False | The clusterId in the request does not match that found on the server |
TRANSACTIONAL_ID_NOT_FOUND | 105 | False | The transactionalId could not be found |
FETCH_SESSION_TOPIC_ID_ERROR | 106 | True | The fetch session encountered inconsistent topic ID usage |
INELIGIBLE_REPLICA | 107 | False | The new ISR contains at least one ineligible replica. |
NEW_LEADER_ELECTED | 108 | False | The AlterPartition request successfully updated the partition state but the leader has changed. |
The following are the numeric codes that the ApiKey in the request can take for each of the below request types.
This section gives details on each of the individual API Messages, their usage, their binary format, and the meaning of their fields.
Headers:
Request Header v0 => request_api_key request_api_version correlation_id
request_api_key => INT16
request_api_version => INT16
correlation_id => INT32
Field |
Description |
---|
request_api_key | The API key of this request. |
request_api_version | The API version of this request. |
correlation_id | The correlation ID of this request. |
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_key | The API key of this request. |
request_api_version | The API version of this request. |
correlation_id | The correlation ID of this request. |
client_id | The client ID string. |
Request Header v2 => request_api_key request_api_version correlation_id client_id TAG_BUFFER
request_api_key => INT16
request_api_version => INT16
correlation_id => INT32
client_id => NULLABLE_STRING
Field |
Description |
---|
request_api_key | The API key of this request. |
request_api_version | The API version of this request. |
correlation_id | The correlation ID of this request. |
client_id | The client ID string. |
_tagged_fields | The tagged fields |
Response Header v0 => correlation_id
correlation_id => INT32
Field |
Description |
---|
correlation_id | The correlation ID of this response. |
Response Header v1 => correlation_id TAG_BUFFER
correlation_id => INT32
Field |
Description |
---|
correlation_id | The correlation ID of this response. |
_tagged_fields | The tagged fields |
Requests:
Produce Request (Version: 0) => acks timeout_ms [topic_data]
acks => INT16
timeout_ms => INT32
topic_data => name [partition_data]
name => STRING
partition_data => index records
index => INT32
records => RECORDS
Field |
Description |
---|
acks | The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR. |
timeout_ms | The timeout to await a response in milliseconds. |
topic_data | Each topic to produce to. |
name | The topic name. |
partition_data | Each partition to produce to. |
index | The partition index. |
records | The record data to be produced. |
Produce Request (Version: 1) => acks timeout_ms [topic_data]
acks => INT16
timeout_ms => INT32
topic_data => name [partition_data]
name => STRING
partition_data => index records
index => INT32
records => RECORDS
Field |
Description |
---|
acks | The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR. |
timeout_ms | The timeout to await a response in milliseconds. |
topic_data | Each topic to produce to. |
name | The topic name. |
partition_data | Each partition to produce to. |
index | The partition index. |
records | The record data to be produced. |
Produce Request (Version: 2) => acks timeout_ms [topic_data]
acks => INT16
timeout_ms => INT32
topic_data => name [partition_data]
name => STRING
partition_data => index records
index => INT32
records => RECORDS
Field |
Description |
---|
acks | The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR. |
timeout_ms | The timeout to await a response in milliseconds. |
topic_data | Each topic to produce to. |
name | The topic name. |
partition_data | Each partition to produce to. |
index | The partition index. |
records | The record data to be produced. |
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
Field |
Description |
---|
transactional_id | The transactional ID, or null if the producer is not transactional. |
acks | The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR. |
timeout_ms | The timeout to await a response in milliseconds. |
topic_data | Each topic to produce to. |
name | The topic name. |
partition_data | Each partition to produce to. |
index | The partition index. |
records | The 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
Field |
Description |
---|
transactional_id | The transactional ID, or null if the producer is not transactional. |
acks | The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR. |
timeout_ms | The timeout to await a response in milliseconds. |
topic_data | Each topic to produce to. |
name | The topic name. |
partition_data | Each partition to produce to. |
index | The partition index. |
records | The 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
Field |
Description |
---|
transactional_id | The transactional ID, or null if the producer is not transactional. |
acks | The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR. |
timeout_ms | The timeout to await a response in milliseconds. |
topic_data | Each topic to produce to. |
name | The topic name. |
partition_data | Each partition to produce to. |
index | The partition index. |
records | The 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
Field |
Description |
---|
transactional_id | The transactional ID, or null if the producer is not transactional. |
acks | The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR. |
timeout_ms | The timeout to await a response in milliseconds. |
topic_data | Each topic to produce to. |
name | The topic name. |
partition_data | Each partition to produce to. |
index | The partition index. |
records | The 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
Field |
Description |
---|
transactional_id | The transactional ID, or null if the producer is not transactional. |
acks | The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR. |
timeout_ms | The timeout to await a response in milliseconds. |
topic_data | Each topic to produce to. |
name | The topic name. |
partition_data | Each partition to produce to. |
index | The partition index. |
records | The 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
Field |
Description |
---|
transactional_id | The transactional ID, or null if the producer is not transactional. |
acks | The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR. |
timeout_ms | The timeout to await a response in milliseconds. |
topic_data | Each topic to produce to. |
name | The topic name. |
partition_data | Each partition to produce to. |
index | The partition index. |
records | The record data to be produced. |
Produce Request (Version: 9) => transactional_id acks timeout_ms [topic_data] TAG_BUFFER
transactional_id => COMPACT_NULLABLE_STRING
acks => INT16
timeout_ms => INT32
topic_data => name [partition_data] TAG_BUFFER
name => COMPACT_STRING
partition_data => index records TAG_BUFFER
index => INT32
records => COMPACT_RECORDS
Field |
Description |
---|
transactional_id | The transactional ID, or null if the producer is not transactional. |
acks | The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR. |
timeout_ms | The timeout to await a response in milliseconds. |
topic_data | Each topic to produce to. |
name | The topic name. |
partition_data | Each partition to produce to. |
index | The partition index. |
records | The record data to be produced. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
Produce Response (Version: 0) => [responses]
responses => name [partition_responses]
name => STRING
partition_responses => index error_code base_offset
index => INT32
error_code => INT16
base_offset => INT64
Field |
Description |
---|
responses | Each produce response |
name | The topic name |
partition_responses | Each partition that we produced to within the topic. |
index | The partition index. |
error_code | The error code, or 0 if there was no error. |
base_offset | The base offset. |
Produce Response (Version: 1) => [responses] throttle_time_ms
responses => name [partition_responses]
name => STRING
partition_responses => index error_code base_offset
index => INT32
error_code => INT16
base_offset => INT64
throttle_time_ms => INT32
Field |
Description |
---|
responses | Each produce response |
name | The topic name |
partition_responses | Each partition that we produced to within the topic. |
index | The partition index. |
error_code | The error code, or 0 if there was no error. |
base_offset | The base offset. |
throttle_time_ms | The 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: 2) => [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
Field |
Description |
---|
responses | Each produce response |
name | The topic name |
partition_responses | Each partition that we produced to within the topic. |
index | The partition index. |
error_code | The error code, or 0 if there was no error. |
base_offset | The base offset. |
log_append_time_ms | The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended. |
throttle_time_ms | The 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: 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
Field |
Description |
---|
responses | Each produce response |
name | The topic name |
partition_responses | Each partition that we produced to within the topic. |
index | The partition index. |
error_code | The error code, or 0 if there was no error. |
base_offset | The base offset. |
log_append_time_ms | The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended. |
throttle_time_ms | The 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
Field |
Description |
---|
responses | Each produce response |
name | The topic name |
partition_responses | Each partition that we produced to within the topic. |
index | The partition index. |
error_code | The error code, or 0 if there was no error. |
base_offset | The base offset. |
log_append_time_ms | The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended. |
throttle_time_ms | The 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
Field |
Description |
---|
responses | Each produce response |
name | The topic name |
partition_responses | Each partition that we produced to within the topic. |
index | The partition index. |
error_code | The error code, or 0 if there was no error. |
base_offset | The base offset. |
log_append_time_ms | The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended. |
log_start_offset | The log start offset. |
throttle_time_ms | The 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
Field |
Description |
---|
responses | Each produce response |
name | The topic name |
partition_responses | Each partition that we produced to within the topic. |
index | The partition index. |
error_code | The error code, or 0 if there was no error. |
base_offset | The base offset. |
log_append_time_ms | The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended. |
log_start_offset | The log start offset. |
throttle_time_ms | The 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
Field |
Description |
---|
responses | Each produce response |
name | The topic name |
partition_responses | Each partition that we produced to within the topic. |
index | The partition index. |
error_code | The error code, or 0 if there was no error. |
base_offset | The base offset. |
log_append_time_ms | The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended. |
log_start_offset | The log start offset. |
throttle_time_ms | The 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
Field |
Description |
---|
responses | Each produce response |
name | The topic name |
partition_responses | Each partition that we produced to within the topic. |
index | The partition index. |
error_code | The error code, or 0 if there was no error. |
base_offset | The base offset. |
log_append_time_ms | The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended. |
log_start_offset | The log start offset. |
record_errors | The batch indices of records that caused the batch to be dropped |
batch_index | The batch index of the record that cause the batch to be dropped |
batch_index_error_message | The error message of the record that caused the batch to be dropped |
error_message | The global error message summarizing the common root cause of the records that caused the batch to be dropped |
throttle_time_ms | The 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 TAG_BUFFER
responses => name [partition_responses] TAG_BUFFER
name => COMPACT_STRING
partition_responses => index error_code base_offset log_append_time_ms log_start_offset [record_errors] error_message TAG_BUFFER
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 TAG_BUFFER
batch_index => INT32
batch_index_error_message => COMPACT_NULLABLE_STRING
error_message => COMPACT_NULLABLE_STRING
throttle_time_ms => INT32
Field |
Description |
---|
responses | Each produce response |
name | The topic name |
partition_responses | Each partition that we produced to within the topic. |
index | The partition index. |
error_code | The error code, or 0 if there was no error. |
base_offset | The base offset. |
log_append_time_ms | The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended. |
log_start_offset | The log start offset. |
record_errors | The batch indices of records that caused the batch to be dropped |
batch_index | The batch index of the record that cause the batch to be dropped |
batch_index_error_message | The error message of the record that caused the batch to be dropped |
_tagged_fields | The tagged fields |
error_message | The global error message summarizing the common root cause of the records that caused the batch to be dropped |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
throttle_time_ms | The 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 | The tagged fields |
Requests:
Fetch Request (Version: 0) => replica_id max_wait_ms min_bytes [topics]
replica_id => INT32
max_wait_ms => INT32
min_bytes => INT32
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset partition_max_bytes
partition => INT32
fetch_offset => INT64
partition_max_bytes => INT32
Field |
Description |
---|
replica_id | The broker ID of the follower, of -1 if this request is from a consumer. |
max_wait_ms | The maximum time in milliseconds to wait for the response. |
min_bytes | The minimum bytes to accumulate in the response. |
topics | The topics to fetch. |
topic | The name of the topic to fetch. |
partitions | The partitions to fetch. |
partition | The partition index. |
fetch_offset | The message offset. |
partition_max_bytes | The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored. |
Fetch Request (Version: 1) => replica_id max_wait_ms min_bytes [topics]
replica_id => INT32
max_wait_ms => INT32
min_bytes => INT32
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset partition_max_bytes
partition => INT32
fetch_offset => INT64
partition_max_bytes => INT32
Field |
Description |
---|
replica_id | The broker ID of the follower, of -1 if this request is from a consumer. |
max_wait_ms | The maximum time in milliseconds to wait for the response. |
min_bytes | The minimum bytes to accumulate in the response. |
topics | The topics to fetch. |
topic | The name of the topic to fetch. |
partitions | The partitions to fetch. |
partition | The partition index. |
fetch_offset | The message offset. |
partition_max_bytes | The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored. |
Fetch Request (Version: 2) => replica_id max_wait_ms min_bytes [topics]
replica_id => INT32
max_wait_ms => INT32
min_bytes => INT32
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset partition_max_bytes
partition => INT32
fetch_offset => INT64
partition_max_bytes => INT32
Field |
Description |
---|
replica_id | The broker ID of the follower, of -1 if this request is from a consumer. |
max_wait_ms | The maximum time in milliseconds to wait for the response. |
min_bytes | The minimum bytes to accumulate in the response. |
topics | The topics to fetch. |
topic | The name of the topic to fetch. |
partitions | The partitions to fetch. |
partition | The partition index. |
fetch_offset | The message offset. |
partition_max_bytes | The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored. |
Fetch Request (Version: 3) => replica_id max_wait_ms min_bytes max_bytes [topics]
replica_id => INT32
max_wait_ms => INT32
min_bytes => INT32
max_bytes => INT32
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset partition_max_bytes
partition => INT32
fetch_offset => INT64
partition_max_bytes => INT32
Field |
Description |
---|
replica_id | The broker ID of the follower, of -1 if this request is from a consumer. |
max_wait_ms | The maximum time in milliseconds to wait for the response. |
min_bytes | The minimum bytes to accumulate in the response. |
max_bytes | The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored. |
topics | The topics to fetch. |
topic | The name of the topic to fetch. |
partitions | The partitions to fetch. |
partition | The partition index. |
fetch_offset | The message offset. |
partition_max_bytes | The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored. |
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
Field |
Description |
---|
replica_id | The broker ID of the follower, of -1 if this request is from a consumer. |
max_wait_ms | The maximum time in milliseconds to wait for the response. |
min_bytes | The minimum bytes to accumulate in the response. |
max_bytes | The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
topics | The topics to fetch. |
topic | The name of the topic to fetch. |
partitions | The partitions to fetch. |
partition | The partition index. |
fetch_offset | The message offset. |
partition_max_bytes | The 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
Field |
Description |
---|
replica_id | The broker ID of the follower, of -1 if this request is from a consumer. |
max_wait_ms | The maximum time in milliseconds to wait for the response. |
min_bytes | The minimum bytes to accumulate in the response. |
max_bytes | The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
topics | The topics to fetch. |
topic | The name of the topic to fetch. |
partitions | The partitions to fetch. |
partition | The partition index. |
fetch_offset | The message offset. |
log_start_offset | The earliest available offset of the follower replica. The field is only used when the request is sent by the follower. |
partition_max_bytes | The 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
Field |
Description |
---|
replica_id | The broker ID of the follower, of -1 if this request is from a consumer. |
max_wait_ms | The maximum time in milliseconds to wait for the response. |
min_bytes | The minimum bytes to accumulate in the response. |
max_bytes | The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
topics | The topics to fetch. |
topic | The name of the topic to fetch. |
partitions | The partitions to fetch. |
partition | The partition index. |
fetch_offset | The message offset. |
log_start_offset | The earliest available offset of the follower replica. The field is only used when the request is sent by the follower. |
partition_max_bytes | The 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
Field |
Description |
---|
replica_id | The broker ID of the follower, of -1 if this request is from a consumer. |
max_wait_ms | The maximum time in milliseconds to wait for the response. |
min_bytes | The minimum bytes to accumulate in the response. |
max_bytes | The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
session_id | The fetch session ID. |
session_epoch | The fetch session epoch, which is used for ordering requests in a session. |
topics | The topics to fetch. |
topic | The name of the topic to fetch. |
partitions | The partitions to fetch. |
partition | The partition index. |
fetch_offset | The message offset. |
log_start_offset | The earliest available offset of the follower replica. The field is only used when the request is sent by the follower. |
partition_max_bytes | The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored. |
forgotten_topics_data | In an incremental fetch request, the partitions to remove. |
topic | The topic name. |
partitions | The 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
Field |
Description |
---|
replica_id | The broker ID of the follower, of -1 if this request is from a consumer. |
max_wait_ms | The maximum time in milliseconds to wait for the response. |
min_bytes | The minimum bytes to accumulate in the response. |
max_bytes | The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
session_id | The fetch session ID. |
session_epoch | The fetch session epoch, which is used for ordering requests in a session. |
topics | The topics to fetch. |
topic | The name of the topic to fetch. |
partitions | The partitions to fetch. |
partition | The partition index. |
fetch_offset | The message offset. |
log_start_offset | The earliest available offset of the follower replica. The field is only used when the request is sent by the follower. |
partition_max_bytes | The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored. |
forgotten_topics_data | In an incremental fetch request, the partitions to remove. |
topic | The topic name. |
partitions | The 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
Field |
Description |
---|
replica_id | The broker ID of the follower, of -1 if this request is from a consumer. |
max_wait_ms | The maximum time in milliseconds to wait for the response. |
min_bytes | The minimum bytes to accumulate in the response. |
max_bytes | The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
session_id | The fetch session ID. |
session_epoch | The fetch session epoch, which is used for ordering requests in a session. |
topics | The topics to fetch. |
topic | The name of the topic to fetch. |
partitions | The partitions to fetch. |
partition | The partition index. |
current_leader_epoch | The current leader epoch of the partition. |
fetch_offset | The message offset. |
log_start_offset | The earliest available offset of the follower replica. The field is only used when the request is sent by the follower. |
partition_max_bytes | The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored. |
forgotten_topics_data | In an incremental fetch request, the partitions to remove. |
topic | The topic name. |
partitions | The 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
Field |
Description |
---|
replica_id | The broker ID of the follower, of -1 if this request is from a consumer. |
max_wait_ms | The maximum time in milliseconds to wait for the response. |
min_bytes | The minimum bytes to accumulate in the response. |
max_bytes | The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
session_id | The fetch session ID. |
session_epoch | The fetch session epoch, which is used for ordering requests in a session. |
topics | The topics to fetch. |
topic | The name of the topic to fetch. |
partitions | The partitions to fetch. |
partition | The partition index. |
current_leader_epoch | The current leader epoch of the partition. |
fetch_offset | The message offset. |
log_start_offset | The earliest available offset of the follower replica. The field is only used when the request is sent by the follower. |
partition_max_bytes | The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored. |
forgotten_topics_data | In an incremental fetch request, the partitions to remove. |
topic | The topic name. |
partitions | The 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
Field |
Description |
---|
replica_id | The broker ID of the follower, of -1 if this request is from a consumer. |
max_wait_ms | The maximum time in milliseconds to wait for the response. |
min_bytes | The minimum bytes to accumulate in the response. |
max_bytes | The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
session_id | The fetch session ID. |
session_epoch | The fetch session epoch, which is used for ordering requests in a session. |
topics | The topics to fetch. |
topic | The name of the topic to fetch. |
partitions | The partitions to fetch. |
partition | The partition index. |
current_leader_epoch | The current leader epoch of the partition. |
fetch_offset | The message offset. |
log_start_offset | The earliest available offset of the follower replica. The field is only used when the request is sent by the follower. |
partition_max_bytes | The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored. |
forgotten_topics_data | In an incremental fetch request, the partitions to remove. |
topic | The topic name. |
partitions | The partitions indexes to forget. |
rack_id | Rack 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 TAG_BUFFER
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] TAG_BUFFER
topic => COMPACT_STRING
partitions => partition current_leader_epoch fetch_offset last_fetched_epoch log_start_offset partition_max_bytes TAG_BUFFER
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] TAG_BUFFER
topic => COMPACT_STRING
partitions => INT32
rack_id => COMPACT_STRING
Field |
Description |
---|
replica_id | The broker ID of the follower, of -1 if this request is from a consumer. |
max_wait_ms | The maximum time in milliseconds to wait for the response. |
min_bytes | The minimum bytes to accumulate in the response. |
max_bytes | The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
session_id | The fetch session ID. |
session_epoch | The fetch session epoch, which is used for ordering requests in a session. |
topics | The topics to fetch. |
topic | The name of the topic to fetch. |
partitions | The partitions to fetch. |
partition | The partition index. |
current_leader_epoch | The current leader epoch of the partition. |
fetch_offset | The message offset. |
last_fetched_epoch | The epoch of the last fetched record or -1 if there is none |
log_start_offset | The earliest available offset of the follower replica. The field is only used when the request is sent by the follower. |
partition_max_bytes | The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
forgotten_topics_data | In an incremental fetch request, the partitions to remove. |
topic | The topic name. |
partitions | The partitions indexes to forget. |
_tagged_fields | The tagged fields |
rack_id | Rack ID of the consumer making this request |
_tagged_fields | The tagged fields |
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 TAG_BUFFER
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] TAG_BUFFER
topic_id => UUID
partitions => partition current_leader_epoch fetch_offset last_fetched_epoch log_start_offset partition_max_bytes TAG_BUFFER
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] TAG_BUFFER
topic_id => UUID
partitions => INT32
rack_id => COMPACT_STRING
Field |
Description |
---|
replica_id | The broker ID of the follower, of -1 if this request is from a consumer. |
max_wait_ms | The maximum time in milliseconds to wait for the response. |
min_bytes | The minimum bytes to accumulate in the response. |
max_bytes | The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
session_id | The fetch session ID. |
session_epoch | The fetch session epoch, which is used for ordering requests in a session. |
topics | The topics to fetch. |
topic_id | The unique topic ID |
partitions | The partitions to fetch. |
partition | The partition index. |
current_leader_epoch | The current leader epoch of the partition. |
fetch_offset | The message offset. |
last_fetched_epoch | The epoch of the last fetched record or -1 if there is none |
log_start_offset | The earliest available offset of the follower replica. The field is only used when the request is sent by the follower. |
partition_max_bytes | The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
forgotten_topics_data | In an incremental fetch request, the partitions to remove. |
topic_id | The unique topic ID |
partitions | The partitions indexes to forget. |
_tagged_fields | The tagged fields |
rack_id | Rack ID of the consumer making this request |
_tagged_fields | The tagged fields |
Responses:
Fetch Response (Version: 0) => [responses]
responses => topic [partitions]
topic => STRING
partitions => partition_index error_code high_watermark records
partition_index => INT32
error_code => INT16
high_watermark => INT64
records => RECORDS
Field |
Description |
---|
responses | The response topics. |
topic | The topic name. |
partitions | The topic partitions. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no fetch error. |
high_watermark | The current high water mark. |
records | The record data. |
Fetch Response (Version: 1) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partitions]
topic => STRING
partitions => partition_index error_code high_watermark records
partition_index => INT32
error_code => INT16
high_watermark => INT64
records => RECORDS
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The response topics. |
topic | The topic name. |
partitions | The topic partitions. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no fetch error. |
high_watermark | The current high water mark. |
records | The record data. |
Fetch Response (Version: 2) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partitions]
topic => STRING
partitions => partition_index error_code high_watermark records
partition_index => INT32
error_code => INT16
high_watermark => INT64
records => RECORDS
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The response topics. |
topic | The topic name. |
partitions | The topic partitions. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no fetch error. |
high_watermark | The current high water mark. |
records | The record data. |
Fetch Response (Version: 3) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partitions]
topic => STRING
partitions => partition_index error_code high_watermark records
partition_index => INT32
error_code => INT16
high_watermark => INT64
records => RECORDS
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The response topics. |
topic | The topic name. |
partitions | The topic partitions. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no fetch error. |
high_watermark | The current high water mark. |
records | The record data. |
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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The response topics. |
topic | The topic name. |
partitions | The topic partitions. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no fetch error. |
high_watermark | The current high water mark. |
last_stable_offset | The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED) |
aborted_transactions | The aborted transactions. |
producer_id | The producer id associated with the aborted transaction. |
first_offset | The first offset in the aborted transaction. |
records | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The response topics. |
topic | The topic name. |
partitions | The topic partitions. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no fetch error. |
high_watermark | The current high water mark. |
last_stable_offset | The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED) |
log_start_offset | The current log start offset. |
aborted_transactions | The aborted transactions. |
producer_id | The producer id associated with the aborted transaction. |
first_offset | The first offset in the aborted transaction. |
records | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The response topics. |
topic | The topic name. |
partitions | The topic partitions. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no fetch error. |
high_watermark | The current high water mark. |
last_stable_offset | The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED) |
log_start_offset | The current log start offset. |
aborted_transactions | The aborted transactions. |
producer_id | The producer id associated with the aborted transaction. |
first_offset | The first offset in the aborted transaction. |
records | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The top level response error code. |
session_id | The fetch session ID, or 0 if this is not part of a fetch session. |
responses | The response topics. |
topic | The topic name. |
partitions | The topic partitions. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no fetch error. |
high_watermark | The current high water mark. |
last_stable_offset | The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED) |
log_start_offset | The current log start offset. |
aborted_transactions | The aborted transactions. |
producer_id | The producer id associated with the aborted transaction. |
first_offset | The first offset in the aborted transaction. |
records | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The top level response error code. |
session_id | The fetch session ID, or 0 if this is not part of a fetch session. |
responses | The response topics. |
topic | The topic name. |
partitions | The topic partitions. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no fetch error. |
high_watermark | The current high water mark. |
last_stable_offset | The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED) |
log_start_offset | The current log start offset. |
aborted_transactions | The aborted transactions. |
producer_id | The producer id associated with the aborted transaction. |
first_offset | The first offset in the aborted transaction. |
records | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The top level response error code. |
session_id | The fetch session ID, or 0 if this is not part of a fetch session. |
responses | The response topics. |
topic | The topic name. |
partitions | The topic partitions. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no fetch error. |
high_watermark | The current high water mark. |
last_stable_offset | The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED) |
log_start_offset | The current log start offset. |
aborted_transactions | The aborted transactions. |
producer_id | The producer id associated with the aborted transaction. |
first_offset | The first offset in the aborted transaction. |
records | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The top level response error code. |
session_id | The fetch session ID, or 0 if this is not part of a fetch session. |
responses | The response topics. |
topic | The topic name. |
partitions | The topic partitions. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no fetch error. |
high_watermark | The current high water mark. |
last_stable_offset | The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED) |
log_start_offset | The current log start offset. |
aborted_transactions | The aborted transactions. |
producer_id | The producer id associated with the aborted transaction. |
first_offset | The first offset in the aborted transaction. |
records | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The top level response error code. |
session_id | The fetch session ID, or 0 if this is not part of a fetch session. |
responses | The response topics. |
topic | The topic name. |
partitions | The topic partitions. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no fetch error. |
high_watermark | The current high water mark. |
last_stable_offset | The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED) |
log_start_offset | The current log start offset. |
aborted_transactions | The aborted transactions. |
producer_id | The producer id associated with the aborted transaction. |
first_offset | The first offset in the aborted transaction. |
preferred_read_replica | The preferred read replica for the consumer to use on its next fetch request |
records | The record data. |
Fetch Response (Version: 12) => throttle_time_ms error_code session_id [responses] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
session_id => INT32
responses => topic [partitions] TAG_BUFFER
topic => COMPACT_STRING
partitions => partition_index error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] preferred_read_replica records TAG_BUFFER
partition_index => INT32
error_code => INT16
high_watermark => INT64
last_stable_offset => INT64
log_start_offset => INT64
aborted_transactions => producer_id first_offset TAG_BUFFER
producer_id => INT64
first_offset => INT64
preferred_read_replica => INT32
records => COMPACT_RECORDS
Field |
Description |
---|
throttle_time_ms | The 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_code | The top level response error code. |
session_id | The fetch session ID, or 0 if this is not part of a fetch session. |
responses | The response topics. |
topic | The topic name. |
partitions | The topic partitions. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no fetch error. |
high_watermark | The current high water mark. |
last_stable_offset | The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED) |
log_start_offset | The current log start offset. |
aborted_transactions | The aborted transactions. |
producer_id | The producer id associated with the aborted transaction. |
first_offset | The first offset in the aborted transaction. |
_tagged_fields | The tagged fields |
preferred_read_replica | The preferred read replica for the consumer to use on its next fetch request |
records | The record data. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Fetch Response (Version: 13) => throttle_time_ms error_code session_id [responses] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
session_id => INT32
responses => topic_id [partitions] TAG_BUFFER
topic_id => UUID
partitions => partition_index error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] preferred_read_replica records TAG_BUFFER
partition_index => INT32
error_code => INT16
high_watermark => INT64
last_stable_offset => INT64
log_start_offset => INT64
aborted_transactions => producer_id first_offset TAG_BUFFER
producer_id => INT64
first_offset => INT64
preferred_read_replica => INT32
records => COMPACT_RECORDS
Field |
Description |
---|
throttle_time_ms | The 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_code | The top level response error code. |
session_id | The fetch session ID, or 0 if this is not part of a fetch session. |
responses | The response topics. |
topic_id | The unique topic ID |
partitions | The topic partitions. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no fetch error. |
high_watermark | The current high water mark. |
last_stable_offset | The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED) |
log_start_offset | The current log start offset. |
aborted_transactions | The aborted transactions. |
producer_id | The producer id associated with the aborted transaction. |
first_offset | The first offset in the aborted transaction. |
_tagged_fields | The tagged fields |
preferred_read_replica | The preferred read replica for the consumer to use on its next fetch request |
records | The record data. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
ListOffsets Request (Version: 0) => replica_id [topics]
replica_id => INT32
topics => name [partitions]
name => STRING
partitions => partition_index timestamp max_num_offsets
partition_index => INT32
timestamp => INT64
max_num_offsets => INT32
Field |
Description |
---|
replica_id | The broker ID of the requestor, or -1 if this request is being made by a normal consumer. |
topics | Each topic in the request. |
name | The topic name. |
partitions | Each partition in the request. |
partition_index | The partition index. |
timestamp | The current timestamp. |
max_num_offsets | The maximum number of offsets to report. |
ListOffsets Request (Version: 1) => replica_id [topics]
replica_id => INT32
topics => name [partitions]
name => STRING
partitions => partition_index timestamp
partition_index => INT32
timestamp => INT64
Field |
Description |
---|
replica_id | The broker ID of the requestor, or -1 if this request is being made by a normal consumer. |
topics | Each topic in the request. |
name | The topic name. |
partitions | Each partition in the request. |
partition_index | The partition index. |
timestamp | The 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
Field |
Description |
---|
replica_id | The broker ID of the requestor, or -1 if this request is being made by a normal consumer. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
topics | Each topic in the request. |
name | The topic name. |
partitions | Each partition in the request. |
partition_index | The partition index. |
timestamp | The 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
Field |
Description |
---|
replica_id | The broker ID of the requestor, or -1 if this request is being made by a normal consumer. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
topics | Each topic in the request. |
name | The topic name. |
partitions | Each partition in the request. |
partition_index | The partition index. |
timestamp | The 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
Field |
Description |
---|
replica_id | The broker ID of the requestor, or -1 if this request is being made by a normal consumer. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
topics | Each topic in the request. |
name | The topic name. |
partitions | Each partition in the request. |
partition_index | The partition index. |
current_leader_epoch | The current leader epoch. |
timestamp | The 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
Field |
Description |
---|
replica_id | The broker ID of the requestor, or -1 if this request is being made by a normal consumer. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
topics | Each topic in the request. |
name | The topic name. |
partitions | Each partition in the request. |
partition_index | The partition index. |
current_leader_epoch | The current leader epoch. |
timestamp | The current timestamp. |
ListOffsets Request (Version: 6) => replica_id isolation_level [topics] TAG_BUFFER
replica_id => INT32
isolation_level => INT8
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index current_leader_epoch timestamp TAG_BUFFER
partition_index => INT32
current_leader_epoch => INT32
timestamp => INT64
Field |
Description |
---|
replica_id | The broker ID of the requestor, or -1 if this request is being made by a normal consumer. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
topics | Each topic in the request. |
name | The topic name. |
partitions | Each partition in the request. |
partition_index | The partition index. |
current_leader_epoch | The current leader epoch. |
timestamp | The current timestamp. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
ListOffsets Request (Version: 7) => replica_id isolation_level [topics] TAG_BUFFER
replica_id => INT32
isolation_level => INT8
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index current_leader_epoch timestamp TAG_BUFFER
partition_index => INT32
current_leader_epoch => INT32
timestamp => INT64
Field |
Description |
---|
replica_id | The broker ID of the requestor, or -1 if this request is being made by a normal consumer. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
topics | Each topic in the request. |
name | The topic name. |
partitions | Each partition in the request. |
partition_index | The partition index. |
current_leader_epoch | The current leader epoch. |
timestamp | The current timestamp. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
ListOffsets Response (Version: 0) => [topics]
topics => name [partitions]
name => STRING
partitions => partition_index error_code [old_style_offsets]
partition_index => INT32
error_code => INT16
old_style_offsets => INT64
Field |
Description |
---|
topics | Each topic in the response. |
name | The topic name |
partitions | Each partition in the response. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
old_style_offsets | The result offsets. |
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
Field |
Description |
---|
topics | Each topic in the response. |
name | The topic name |
partitions | Each partition in the response. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
timestamp | The timestamp associated with the returned offset. |
offset | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Each topic in the response. |
name | The topic name |
partitions | Each partition in the response. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
timestamp | The timestamp associated with the returned offset. |
offset | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Each topic in the response. |
name | The topic name |
partitions | Each partition in the response. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
timestamp | The timestamp associated with the returned offset. |
offset | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Each topic in the response. |
name | The topic name |
partitions | Each partition in the response. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
timestamp | The timestamp associated with the returned offset. |
offset | The returned offset. |
leader_epoch | |
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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Each topic in the response. |
name | The topic name |
partitions | Each partition in the response. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
timestamp | The timestamp associated with the returned offset. |
offset | The returned offset. |
leader_epoch | |
ListOffsets Response (Version: 6) => throttle_time_ms [topics] TAG_BUFFER
throttle_time_ms => INT32
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index error_code timestamp offset leader_epoch TAG_BUFFER
partition_index => INT32
error_code => INT16
timestamp => INT64
offset => INT64
leader_epoch => INT32
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Each topic in the response. |
name | The topic name |
partitions | Each partition in the response. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
timestamp | The timestamp associated with the returned offset. |
offset | The returned offset. |
leader_epoch | |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
ListOffsets Response (Version: 7) => throttle_time_ms [topics] TAG_BUFFER
throttle_time_ms => INT32
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index error_code timestamp offset leader_epoch TAG_BUFFER
partition_index => INT32
error_code => INT16
timestamp => INT64
offset => INT64
leader_epoch => INT32
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Each topic in the response. |
name | The topic name |
partitions | Each partition in the response. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
timestamp | The timestamp associated with the returned offset. |
offset | The returned offset. |
leader_epoch | |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
Metadata Request (Version: 0) => [topics]
topics => name
name => STRING
Field |
Description |
---|
topics | The topics to fetch metadata for. |
name | The topic name. |
Metadata Request (Version: 1) => [topics]
topics => name
name => STRING
Field |
Description |
---|
topics | The topics to fetch metadata for. |
name | The topic name. |
Metadata Request (Version: 2) => [topics]
topics => name
name => STRING
Field |
Description |
---|
topics | The topics to fetch metadata for. |
name | The topic name. |
Metadata Request (Version: 3) => [topics]
topics => name
name => STRING
Field |
Description |
---|
topics | The topics to fetch metadata for. |
name | The topic name. |
Metadata Request (Version: 4) => [topics] allow_auto_topic_creation
topics => name
name => STRING
allow_auto_topic_creation => BOOLEAN
Field |
Description |
---|
topics | The topics to fetch metadata for. |
name | The topic name. |
allow_auto_topic_creation | If 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
Field |
Description |
---|
topics | The topics to fetch metadata for. |
name | The topic name. |
allow_auto_topic_creation | If 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
Field |
Description |
---|
topics | The topics to fetch metadata for. |
name | The topic name. |
allow_auto_topic_creation | If 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
Field |
Description |
---|
topics | The topics to fetch metadata for. |
name | The topic name. |
allow_auto_topic_creation | If 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
Field |
Description |
---|
topics | The topics to fetch metadata for. |
name | The topic name. |
allow_auto_topic_creation | If 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_operations | Whether to include cluster authorized operations. |
include_topic_authorized_operations | Whether to include topic authorized operations. |
Metadata Request (Version: 9) => [topics] allow_auto_topic_creation include_cluster_authorized_operations include_topic_authorized_operations TAG_BUFFER
topics => name TAG_BUFFER
name => COMPACT_STRING
allow_auto_topic_creation => BOOLEAN
include_cluster_authorized_operations => BOOLEAN
include_topic_authorized_operations => BOOLEAN
Field |
Description |
---|
topics | The topics to fetch metadata for. |
name | The topic name. |
_tagged_fields | The tagged fields |
allow_auto_topic_creation | If 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_operations | Whether to include cluster authorized operations. |
include_topic_authorized_operations | Whether to include topic authorized operations. |
_tagged_fields | The tagged fields |
Metadata Request (Version: 10) => [topics] allow_auto_topic_creation include_cluster_authorized_operations include_topic_authorized_operations TAG_BUFFER
topics => topic_id name TAG_BUFFER
topic_id => UUID
name => COMPACT_NULLABLE_STRING
allow_auto_topic_creation => BOOLEAN
include_cluster_authorized_operations => BOOLEAN
include_topic_authorized_operations => BOOLEAN
Field |
Description |
---|
topics | The topics to fetch metadata for. |
topic_id | The topic id. |
name | The topic name. |
_tagged_fields | The tagged fields |
allow_auto_topic_creation | If 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_operations | Whether to include cluster authorized operations. |
include_topic_authorized_operations | Whether to include topic authorized operations. |
_tagged_fields | The tagged fields |
Metadata Request (Version: 11) => [topics] allow_auto_topic_creation include_topic_authorized_operations TAG_BUFFER
topics => topic_id name TAG_BUFFER
topic_id => UUID
name => COMPACT_NULLABLE_STRING
allow_auto_topic_creation => BOOLEAN
include_topic_authorized_operations => BOOLEAN
Field |
Description |
---|
topics | The topics to fetch metadata for. |
topic_id | The topic id. |
name | The topic name. |
_tagged_fields | The tagged fields |
allow_auto_topic_creation | If 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_operations | Whether to include topic authorized operations. |
_tagged_fields | The tagged fields |
Metadata Request (Version: 12) => [topics] allow_auto_topic_creation include_topic_authorized_operations TAG_BUFFER
topics => topic_id name TAG_BUFFER
topic_id => UUID
name => COMPACT_NULLABLE_STRING
allow_auto_topic_creation => BOOLEAN
include_topic_authorized_operations => BOOLEAN
Field |
Description |
---|
topics | The topics to fetch metadata for. |
topic_id | The topic id. |
name | The topic name. |
_tagged_fields | The tagged fields |
allow_auto_topic_creation | If 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_operations | Whether to include topic authorized operations. |
_tagged_fields | The 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
Field |
Description |
---|
brokers | Each broker in the response. |
node_id | The broker ID. |
host | The broker hostname. |
port | The broker port. |
topics | Each topic in the response. |
error_code | The topic error, or 0 if there was no error. |
name | The topic name. |
partitions | Each partition in the topic. |
error_code | The partition error, or 0 if there was no error. |
partition_index | The partition index. |
leader_id | The ID of the leader broker. |
replica_nodes | The set of all nodes that host this partition. |
isr_nodes | The 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
Field |
Description |
---|
brokers | Each broker in the response. |
node_id | The broker ID. |
host | The broker hostname. |
port | The broker port. |
rack | The rack of the broker, or null if it has not been assigned to a rack. |
controller_id | The ID of the controller broker. |
topics | Each topic in the response. |
error_code | The topic error, or 0 if there was no error. |
name | The topic name. |
is_internal | True if the topic is internal. |
partitions | Each partition in the topic. |
error_code | The partition error, or 0 if there was no error. |
partition_index | The partition index. |
leader_id | The ID of the leader broker. |
replica_nodes | The set of all nodes that host this partition. |
isr_nodes | The 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
Field |
Description |
---|
brokers | Each broker in the response. |
node_id | The broker ID. |
host | The broker hostname. |
port | The broker port. |
rack | The rack of the broker, or null if it has not been assigned to a rack. |
cluster_id | The cluster ID that responding broker belongs to. |
controller_id | The ID of the controller broker. |
topics | Each topic in the response. |
error_code | The topic error, or 0 if there was no error. |
name | The topic name. |
is_internal | True if the topic is internal. |
partitions | Each partition in the topic. |
error_code | The partition error, or 0 if there was no error. |
partition_index | The partition index. |
leader_id | The ID of the leader broker. |
replica_nodes | The set of all nodes that host this partition. |
isr_nodes | The set of nodes that are in sync with the leader for this partition. |
Metadata Response (Version: 3) => throttle_time_ms [brokers] cluster_id controller_id [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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
brokers | Each broker in the response. |
node_id | The broker ID. |
host | The broker hostname. |
port | The broker port. |
rack | The rack of the broker, or null if it has not been assigned to a rack. |
cluster_id | The cluster ID that responding broker belongs to. |
controller_id | The ID of the controller broker. |
topics | Each topic in the response. |
error_code | The topic error, or 0 if there was no error. |
name | The topic name. |
is_internal | True if the topic is internal. |
partitions | Each partition in the topic. |
error_code | The partition error, or 0 if there was no error. |
partition_index | The partition index. |
leader_id | The ID of the leader broker. |
replica_nodes | The set of all nodes that host this partition. |
isr_nodes | The set of nodes that are in sync with the leader for this partition. |
Metadata Response (Version: 4) => throttle_time_ms [brokers] cluster_id controller_id [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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
brokers | Each broker in the response. |
node_id | The broker ID. |
host | The broker hostname. |
port | The broker port. |
rack | The rack of the broker, or null if it has not been assigned to a rack. |
cluster_id | The cluster ID that responding broker belongs to. |
controller_id | The ID of the controller broker. |
topics | Each topic in the response. |
error_code | The topic error, or 0 if there was no error. |
name | The topic name. |
is_internal | True if the topic is internal. |
partitions | Each partition in the topic. |
error_code | The partition error, or 0 if there was no error. |
partition_index | The partition index. |
leader_id | The ID of the leader broker. |
replica_nodes | The set of all nodes that host this partition. |
isr_nodes | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
brokers | Each broker in the response. |
node_id | The broker ID. |
host | The broker hostname. |
port | The broker port. |
rack | The rack of the broker, or null if it has not been assigned to a rack. |
cluster_id | The cluster ID that responding broker belongs to. |
controller_id | The ID of the controller broker. |
topics | Each topic in the response. |
error_code | The topic error, or 0 if there was no error. |
name | The topic name. |
is_internal | True if the topic is internal. |
partitions | Each partition in the topic. |
error_code | The partition error, or 0 if there was no error. |
partition_index | The partition index. |
leader_id | The ID of the leader broker. |
replica_nodes | The set of all nodes that host this partition. |
isr_nodes | The set of nodes that are in sync with the leader for this partition. |
offline_replicas | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
brokers | Each broker in the response. |
node_id | The broker ID. |
host | The broker hostname. |
port | The broker port. |
rack | The rack of the broker, or null if it has not been assigned to a rack. |
cluster_id | The cluster ID that responding broker belongs to. |
controller_id | The ID of the controller broker. |
topics | Each topic in the response. |
error_code | The topic error, or 0 if there was no error. |
name | The topic name. |
is_internal | True if the topic is internal. |
partitions | Each partition in the topic. |
error_code | The partition error, or 0 if there was no error. |
partition_index | The partition index. |
leader_id | The ID of the leader broker. |
replica_nodes | The set of all nodes that host this partition. |
isr_nodes | The set of nodes that are in sync with the leader for this partition. |
offline_replicas | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
brokers | Each broker in the response. |
node_id | The broker ID. |
host | The broker hostname. |
port | The broker port. |
rack | The rack of the broker, or null if it has not been assigned to a rack. |
cluster_id | The cluster ID that responding broker belongs to. |
controller_id | The ID of the controller broker. |
topics | Each topic in the response. |
error_code | The topic error, or 0 if there was no error. |
name | The topic name. |
is_internal | True if the topic is internal. |
partitions | Each partition in the topic. |
error_code | The partition error, or 0 if there was no error. |
partition_index | The partition index. |
leader_id | The ID of the leader broker. |
leader_epoch | The leader epoch of this partition. |
replica_nodes | The set of all nodes that host this partition. |
isr_nodes | The set of nodes that are in sync with the leader for this partition. |
offline_replicas | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
brokers | Each broker in the response. |
node_id | The broker ID. |
host | The broker hostname. |
port | The broker port. |
rack | The rack of the broker, or null if it has not been assigned to a rack. |
cluster_id | The cluster ID that responding broker belongs to. |
controller_id | The ID of the controller broker. |
topics | Each topic in the response. |
error_code | The topic error, or 0 if there was no error. |
name | The topic name. |
is_internal | True if the topic is internal. |
partitions | Each partition in the topic. |
error_code | The partition error, or 0 if there was no error. |
partition_index | The partition index. |
leader_id | The ID of the leader broker. |
leader_epoch | The leader epoch of this partition. |
replica_nodes | The set of all nodes that host this partition. |
isr_nodes | The set of nodes that are in sync with the leader for this partition. |
offline_replicas | The set of offline replicas of this partition. |
topic_authorized_operations | 32-bit bitfield to represent authorized operations for this topic. |
cluster_authorized_operations | 32-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 TAG_BUFFER
throttle_time_ms => INT32
brokers => node_id host port rack TAG_BUFFER
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 TAG_BUFFER
error_code => INT16
name => COMPACT_STRING
is_internal => BOOLEAN
partitions => error_code partition_index leader_id leader_epoch [replica_nodes] [isr_nodes] [offline_replicas] TAG_BUFFER
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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
brokers | Each broker in the response. |
node_id | The broker ID. |
host | The broker hostname. |
port | The broker port. |
rack | The rack of the broker, or null if it has not been assigned to a rack. |
_tagged_fields | The tagged fields |
cluster_id | The cluster ID that responding broker belongs to. |
controller_id | The ID of the controller broker. |
topics | Each topic in the response. |
error_code | The topic error, or 0 if there was no error. |
name | The topic name. |
is_internal | True if the topic is internal. |
partitions | Each partition in the topic. |
error_code | The partition error, or 0 if there was no error. |
partition_index | The partition index. |
leader_id | The ID of the leader broker. |
leader_epoch | The leader epoch of this partition. |
replica_nodes | The set of all nodes that host this partition. |
isr_nodes | The set of nodes that are in sync with the leader for this partition. |
offline_replicas | The set of offline replicas of this partition. |
_tagged_fields | The tagged fields |
topic_authorized_operations | 32-bit bitfield to represent authorized operations for this topic. |
_tagged_fields | The tagged fields |
cluster_authorized_operations | 32-bit bitfield to represent authorized operations for this cluster. |
_tagged_fields | The tagged fields |
Metadata Response (Version: 10) => throttle_time_ms [brokers] cluster_id controller_id [topics] cluster_authorized_operations TAG_BUFFER
throttle_time_ms => INT32
brokers => node_id host port rack TAG_BUFFER
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 TAG_BUFFER
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] TAG_BUFFER
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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
brokers | Each broker in the response. |
node_id | The broker ID. |
host | The broker hostname. |
port | The broker port. |
rack | The rack of the broker, or null if it has not been assigned to a rack. |
_tagged_fields | The tagged fields |
cluster_id | The cluster ID that responding broker belongs to. |
controller_id | The ID of the controller broker. |
topics | Each topic in the response. |
error_code | The topic error, or 0 if there was no error. |
name | The topic name. |
topic_id | The topic id. |
is_internal | True if the topic is internal. |
partitions | Each partition in the topic. |
error_code | The partition error, or 0 if there was no error. |
partition_index | The partition index. |
leader_id | The ID of the leader broker. |
leader_epoch | The leader epoch of this partition. |
replica_nodes | The set of all nodes that host this partition. |
isr_nodes | The set of nodes that are in sync with the leader for this partition. |
offline_replicas | The set of offline replicas of this partition. |
_tagged_fields | The tagged fields |
topic_authorized_operations | 32-bit bitfield to represent authorized operations for this topic. |
_tagged_fields | The tagged fields |
cluster_authorized_operations | 32-bit bitfield to represent authorized operations for this cluster. |
_tagged_fields | The tagged fields |
Metadata Response (Version: 11) => throttle_time_ms [brokers] cluster_id controller_id [topics] TAG_BUFFER
throttle_time_ms => INT32
brokers => node_id host port rack TAG_BUFFER
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 TAG_BUFFER
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] TAG_BUFFER
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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
brokers | Each broker in the response. |
node_id | The broker ID. |
host | The broker hostname. |
port | The broker port. |
rack | The rack of the broker, or null if it has not been assigned to a rack. |
_tagged_fields | The tagged fields |
cluster_id | The cluster ID that responding broker belongs to. |
controller_id | The ID of the controller broker. |
topics | Each topic in the response. |
error_code | The topic error, or 0 if there was no error. |
name | The topic name. |
topic_id | The topic id. |
is_internal | True if the topic is internal. |
partitions | Each partition in the topic. |
error_code | The partition error, or 0 if there was no error. |
partition_index | The partition index. |
leader_id | The ID of the leader broker. |
leader_epoch | The leader epoch of this partition. |
replica_nodes | The set of all nodes that host this partition. |
isr_nodes | The set of nodes that are in sync with the leader for this partition. |
offline_replicas | The set of offline replicas of this partition. |
_tagged_fields | The tagged fields |
topic_authorized_operations | 32-bit bitfield to represent authorized operations for this topic. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Metadata Response (Version: 12) => throttle_time_ms [brokers] cluster_id controller_id [topics] TAG_BUFFER
throttle_time_ms => INT32
brokers => node_id host port rack TAG_BUFFER
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 TAG_BUFFER
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] TAG_BUFFER
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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
brokers | Each broker in the response. |
node_id | The broker ID. |
host | The broker hostname. |
port | The broker port. |
rack | The rack of the broker, or null if it has not been assigned to a rack. |
_tagged_fields | The tagged fields |
cluster_id | The cluster ID that responding broker belongs to. |
controller_id | The ID of the controller broker. |
topics | Each topic in the response. |
error_code | The topic error, or 0 if there was no error. |
name | The topic name. |
topic_id | The topic id. |
is_internal | True if the topic is internal. |
partitions | Each partition in the topic. |
error_code | The partition error, or 0 if there was no error. |
partition_index | The partition index. |
leader_id | The ID of the leader broker. |
leader_epoch | The leader epoch of this partition. |
replica_nodes | The set of all nodes that host this partition. |
isr_nodes | The set of nodes that are in sync with the leader for this partition. |
offline_replicas | The set of offline replicas of this partition. |
_tagged_fields | The tagged fields |
topic_authorized_operations | 32-bit bitfield to represent authorized operations for this topic. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
LeaderAndIsr Request (Version: 0) => controller_id controller_epoch [ungrouped_partition_states] [live_leaders]
controller_id => INT32
controller_epoch => INT32
ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] partition_epoch [replicas]
topic_name => STRING
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
partition_epoch => INT32
replicas => INT32
live_leaders => broker_id host_name port
broker_id => INT32
host_name => STRING
port => INT32
Field |
Description |
---|
controller_id | The current controller ID. |
controller_epoch | The current controller epoch. |
ungrouped_partition_states | The state of each partition, in a v0 or v1 message. |
topic_name | The topic name. This is only present in v0 or v1. |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The broker ID of the leader. |
leader_epoch | The leader epoch. |
isr | The in-sync replica IDs. |
partition_epoch | The current epoch for the partition. The epoch is a monotonically increasing value which is incremented after every partition change. (Since the LeaderAndIsr request is only used by the legacy controller, this corresponds to the zkVersion) |
replicas | The replica IDs. |
live_leaders | The current live leaders. |
broker_id | The leader's broker ID. |
host_name | The leader's hostname. |
port | The leader's port. |
LeaderAndIsr Request (Version: 1) => controller_id controller_epoch [ungrouped_partition_states] [live_leaders]
controller_id => INT32
controller_epoch => INT32
ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] partition_epoch [replicas] is_new
topic_name => STRING
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
partition_epoch => INT32
replicas => INT32
is_new => BOOLEAN
live_leaders => broker_id host_name port
broker_id => INT32
host_name => STRING
port => INT32
Field |
Description |
---|
controller_id | The current controller ID. |
controller_epoch | The current controller epoch. |
ungrouped_partition_states | The state of each partition, in a v0 or v1 message. |
topic_name | The topic name. This is only present in v0 or v1. |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The broker ID of the leader. |
leader_epoch | The leader epoch. |
isr | The in-sync replica IDs. |
partition_epoch | The current epoch for the partition. The epoch is a monotonically increasing value which is incremented after every partition change. (Since the LeaderAndIsr request is only used by the legacy controller, this corresponds to the zkVersion) |
replicas | The replica IDs. |
is_new | Whether the replica should have existed on the broker or not. |
live_leaders | The current live leaders. |
broker_id | The leader's broker ID. |
host_name | The leader's hostname. |
port | The leader's port. |
LeaderAndIsr Request (Version: 2) => controller_id controller_epoch broker_epoch [topic_states] [live_leaders]
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
topic_states => topic_name [partition_states]
topic_name => STRING
partition_states => partition_index controller_epoch leader leader_epoch [isr] partition_epoch [replicas] is_new
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
partition_epoch => INT32
replicas => INT32
is_new => BOOLEAN
live_leaders => broker_id host_name port
broker_id => INT32
host_name => STRING
port => INT32
Field |
Description |
---|
controller_id | The current controller ID. |
controller_epoch | The current controller epoch. |
broker_epoch | The current broker epoch. |
topic_states | Each topic. |
topic_name | The topic name. |
partition_states | The state of each partition |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The broker ID of the leader. |
leader_epoch | The leader epoch. |
isr | The in-sync replica IDs. |
partition_epoch | The current epoch for the partition. The epoch is a monotonically increasing value which is incremented after every partition change. (Since the LeaderAndIsr request is only used by the legacy controller, this corresponds to the zkVersion) |
replicas | The replica IDs. |
is_new | Whether the replica should have existed on the broker or not. |
live_leaders | The current live leaders. |
broker_id | The leader's broker ID. |
host_name | The leader's hostname. |
port | The leader's port. |
LeaderAndIsr Request (Version: 3) => controller_id controller_epoch broker_epoch [topic_states] [live_leaders]
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
topic_states => topic_name [partition_states]
topic_name => STRING
partition_states => partition_index controller_epoch leader leader_epoch [isr] partition_epoch [replicas] [adding_replicas] [removing_replicas] is_new
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
partition_epoch => INT32
replicas => INT32
adding_replicas => INT32
removing_replicas => INT32
is_new => BOOLEAN
live_leaders => broker_id host_name port
broker_id => INT32
host_name => STRING
port => INT32
Field |
Description |
---|
controller_id | The current controller ID. |
controller_epoch | The current controller epoch. |
broker_epoch | The current broker epoch. |
topic_states | Each topic. |
topic_name | The topic name. |
partition_states | The state of each partition |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The broker ID of the leader. |
leader_epoch | The leader epoch. |
isr | The in-sync replica IDs. |
partition_epoch | The current epoch for the partition. The epoch is a monotonically increasing value which is incremented after every partition change. (Since the LeaderAndIsr request is only used by the legacy controller, this corresponds to the zkVersion) |
replicas | The replica IDs. |
adding_replicas | The replica IDs that we are adding this partition to, or null if no replicas are being added. |
removing_replicas | The replica IDs that we are removing this partition from, or null if no replicas are being removed. |
is_new | Whether the replica should have existed on the broker or not. |
live_leaders | The current live leaders. |
broker_id | The leader's broker ID. |
host_name | The leader's hostname. |
port | The leader's port. |
LeaderAndIsr Request (Version: 4) => controller_id controller_epoch broker_epoch [topic_states] [live_leaders] TAG_BUFFER
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
topic_states => topic_name [partition_states] TAG_BUFFER
topic_name => COMPACT_STRING
partition_states => partition_index controller_epoch leader leader_epoch [isr] partition_epoch [replicas] [adding_replicas] [removing_replicas] is_new TAG_BUFFER
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
partition_epoch => INT32
replicas => INT32
adding_replicas => INT32
removing_replicas => INT32
is_new => BOOLEAN
live_leaders => broker_id host_name port TAG_BUFFER
broker_id => INT32
host_name => COMPACT_STRING
port => INT32
Field |
Description |
---|
controller_id | The current controller ID. |
controller_epoch | The current controller epoch. |
broker_epoch | The current broker epoch. |
topic_states | Each topic. |
topic_name | The topic name. |
partition_states | The state of each partition |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The broker ID of the leader. |
leader_epoch | The leader epoch. |
isr | The in-sync replica IDs. |
partition_epoch | The current epoch for the partition. The epoch is a monotonically increasing value which is incremented after every partition change. (Since the LeaderAndIsr request is only used by the legacy controller, this corresponds to the zkVersion) |
replicas | The replica IDs. |
adding_replicas | The replica IDs that we are adding this partition to, or null if no replicas are being added. |
removing_replicas | The replica IDs that we are removing this partition from, or null if no replicas are being removed. |
is_new | Whether the replica should have existed on the broker or not. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
live_leaders | The current live leaders. |
broker_id | The leader's broker ID. |
host_name | The leader's hostname. |
port | The leader's port. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
LeaderAndIsr Request (Version: 5) => controller_id controller_epoch broker_epoch type [topic_states] [live_leaders] TAG_BUFFER
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
type => INT8
topic_states => topic_name topic_id [partition_states] TAG_BUFFER
topic_name => COMPACT_STRING
topic_id => UUID
partition_states => partition_index controller_epoch leader leader_epoch [isr] partition_epoch [replicas] [adding_replicas] [removing_replicas] is_new TAG_BUFFER
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
partition_epoch => INT32
replicas => INT32
adding_replicas => INT32
removing_replicas => INT32
is_new => BOOLEAN
live_leaders => broker_id host_name port TAG_BUFFER
broker_id => INT32
host_name => COMPACT_STRING
port => INT32
Field |
Description |
---|
controller_id | The current controller ID. |
controller_epoch | The current controller epoch. |
broker_epoch | The current broker epoch. |
type | The type that indicates whether all topics are included in the request |
topic_states | Each topic. |
topic_name | The topic name. |
topic_id | The unique topic ID. |
partition_states | The state of each partition |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The broker ID of the leader. |
leader_epoch | The leader epoch. |
isr | The in-sync replica IDs. |
partition_epoch | The current epoch for the partition. The epoch is a monotonically increasing value which is incremented after every partition change. (Since the LeaderAndIsr request is only used by the legacy controller, this corresponds to the zkVersion) |
replicas | The replica IDs. |
adding_replicas | The replica IDs that we are adding this partition to, or null if no replicas are being added. |
removing_replicas | The replica IDs that we are removing this partition from, or null if no replicas are being removed. |
is_new | Whether the replica should have existed on the broker or not. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
live_leaders | The current live leaders. |
broker_id | The leader's broker ID. |
host_name | The leader's hostname. |
port | The leader's port. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
LeaderAndIsr Request (Version: 6) => controller_id controller_epoch broker_epoch type [topic_states] [live_leaders] TAG_BUFFER
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
type => INT8
topic_states => topic_name topic_id [partition_states] TAG_BUFFER
topic_name => COMPACT_STRING
topic_id => UUID
partition_states => partition_index controller_epoch leader leader_epoch [isr] partition_epoch [replicas] [adding_replicas] [removing_replicas] is_new leader_recovery_state TAG_BUFFER
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
partition_epoch => INT32
replicas => INT32
adding_replicas => INT32
removing_replicas => INT32
is_new => BOOLEAN
leader_recovery_state => INT8
live_leaders => broker_id host_name port TAG_BUFFER
broker_id => INT32
host_name => COMPACT_STRING
port => INT32
Field |
Description |
---|
controller_id | The current controller ID. |
controller_epoch | The current controller epoch. |
broker_epoch | The current broker epoch. |
type | The type that indicates whether all topics are included in the request |
topic_states | Each topic. |
topic_name | The topic name. |
topic_id | The unique topic ID. |
partition_states | The state of each partition |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The broker ID of the leader. |
leader_epoch | The leader epoch. |
isr | The in-sync replica IDs. |
partition_epoch | The current epoch for the partition. The epoch is a monotonically increasing value which is incremented after every partition change. (Since the LeaderAndIsr request is only used by the legacy controller, this corresponds to the zkVersion) |
replicas | The replica IDs. |
adding_replicas | The replica IDs that we are adding this partition to, or null if no replicas are being added. |
removing_replicas | The replica IDs that we are removing this partition from, or null if no replicas are being removed. |
is_new | Whether the replica should have existed on the broker or not. |
leader_recovery_state | 1 if the partition is recovering from an unclean leader election; 0 otherwise. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
live_leaders | The current live leaders. |
broker_id | The leader's broker ID. |
host_name | The leader's hostname. |
port | The leader's port. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
LeaderAndIsr Request (Version: 7) => controller_id is_kraft_controller controller_epoch broker_epoch type [topic_states] [live_leaders] TAG_BUFFER
controller_id => INT32
is_kraft_controller => BOOLEAN
controller_epoch => INT32
broker_epoch => INT64
type => INT8
topic_states => topic_name topic_id [partition_states] TAG_BUFFER
topic_name => COMPACT_STRING
topic_id => UUID
partition_states => partition_index controller_epoch leader leader_epoch [isr] partition_epoch [replicas] [adding_replicas] [removing_replicas] is_new leader_recovery_state TAG_BUFFER
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
partition_epoch => INT32
replicas => INT32
adding_replicas => INT32
removing_replicas => INT32
is_new => BOOLEAN
leader_recovery_state => INT8
live_leaders => broker_id host_name port TAG_BUFFER
broker_id => INT32
host_name => COMPACT_STRING
port => INT32
Field |
Description |
---|
controller_id | The current controller ID. |
is_kraft_controller | If KRaft controller id is used during migration. See KIP-866 |
controller_epoch | The current controller epoch. |
broker_epoch | The current broker epoch. |
type | The type that indicates whether all topics are included in the request |
topic_states | Each topic. |
topic_name | The topic name. |
topic_id | The unique topic ID. |
partition_states | The state of each partition |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The broker ID of the leader. |
leader_epoch | The leader epoch. |
isr | The in-sync replica IDs. |
partition_epoch | The current epoch for the partition. The epoch is a monotonically increasing value which is incremented after every partition change. (Since the LeaderAndIsr request is only used by the legacy controller, this corresponds to the zkVersion) |
replicas | The replica IDs. |
adding_replicas | The replica IDs that we are adding this partition to, or null if no replicas are being added. |
removing_replicas | The replica IDs that we are removing this partition from, or null if no replicas are being removed. |
is_new | Whether the replica should have existed on the broker or not. |
leader_recovery_state | 1 if the partition is recovering from an unclean leader election; 0 otherwise. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
live_leaders | The current live leaders. |
broker_id | The leader's broker ID. |
host_name | The leader's hostname. |
port | The leader's port. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
LeaderAndIsr Response (Version: 0) => error_code [partition_errors]
error_code => INT16
partition_errors => topic_name partition_index error_code
topic_name => STRING
partition_index => INT32
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
partition_errors | Each partition in v0 to v4 message. |
topic_name | The topic name. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
LeaderAndIsr Response (Version: 1) => error_code [partition_errors]
error_code => INT16
partition_errors => topic_name partition_index error_code
topic_name => STRING
partition_index => INT32
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
partition_errors | Each partition in v0 to v4 message. |
topic_name | The topic name. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
LeaderAndIsr Response (Version: 2) => error_code [partition_errors]
error_code => INT16
partition_errors => topic_name partition_index error_code
topic_name => STRING
partition_index => INT32
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
partition_errors | Each partition in v0 to v4 message. |
topic_name | The topic name. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
LeaderAndIsr Response (Version: 3) => error_code [partition_errors]
error_code => INT16
partition_errors => topic_name partition_index error_code
topic_name => STRING
partition_index => INT32
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
partition_errors | Each partition in v0 to v4 message. |
topic_name | The topic name. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
LeaderAndIsr Response (Version: 4) => error_code [partition_errors] TAG_BUFFER
error_code => INT16
partition_errors => topic_name partition_index error_code TAG_BUFFER
topic_name => COMPACT_STRING
partition_index => INT32
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
partition_errors | Each partition in v0 to v4 message. |
topic_name | The topic name. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
LeaderAndIsr Response (Version: 5) => error_code [topics] TAG_BUFFER
error_code => INT16
topics => topic_id [partition_errors] TAG_BUFFER
topic_id => UUID
partition_errors => partition_index error_code TAG_BUFFER
partition_index => INT32
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
topics | Each topic |
topic_id | The unique topic ID |
partition_errors | Each partition. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
LeaderAndIsr Response (Version: 6) => error_code [topics] TAG_BUFFER
error_code => INT16
topics => topic_id [partition_errors] TAG_BUFFER
topic_id => UUID
partition_errors => partition_index error_code TAG_BUFFER
partition_index => INT32
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
topics | Each topic |
topic_id | The unique topic ID |
partition_errors | Each partition. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
LeaderAndIsr Response (Version: 7) => error_code [topics] TAG_BUFFER
error_code => INT16
topics => topic_id [partition_errors] TAG_BUFFER
topic_id => UUID
partition_errors => partition_index error_code TAG_BUFFER
partition_index => INT32
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
topics | Each topic |
topic_id | The unique topic ID |
partition_errors | Each partition. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
StopReplica Request (Version: 0) => controller_id controller_epoch delete_partitions [ungrouped_partitions]
controller_id => INT32
controller_epoch => INT32
delete_partitions => BOOLEAN
ungrouped_partitions => topic_name partition_index
topic_name => STRING
partition_index => INT32
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
delete_partitions | Whether these partitions should be deleted. |
ungrouped_partitions | The partitions to stop. |
topic_name | The topic name. |
partition_index | The partition index. |
StopReplica Request (Version: 1) => controller_id controller_epoch broker_epoch delete_partitions [topics]
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
delete_partitions => BOOLEAN
topics => name [partition_indexes]
name => STRING
partition_indexes => INT32
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
broker_epoch | The broker epoch. |
delete_partitions | Whether these partitions should be deleted. |
topics | The topics to stop. |
name | The topic name. |
partition_indexes | The partition indexes. |
StopReplica Request (Version: 2) => controller_id controller_epoch broker_epoch delete_partitions [topics] TAG_BUFFER
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
delete_partitions => BOOLEAN
topics => name [partition_indexes] TAG_BUFFER
name => COMPACT_STRING
partition_indexes => INT32
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
broker_epoch | The broker epoch. |
delete_partitions | Whether these partitions should be deleted. |
topics | The topics to stop. |
name | The topic name. |
partition_indexes | The partition indexes. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
StopReplica Request (Version: 3) => controller_id controller_epoch broker_epoch [topic_states] TAG_BUFFER
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
topic_states => topic_name [partition_states] TAG_BUFFER
topic_name => COMPACT_STRING
partition_states => partition_index leader_epoch delete_partition TAG_BUFFER
partition_index => INT32
leader_epoch => INT32
delete_partition => BOOLEAN
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
broker_epoch | The broker epoch. |
topic_states | Each topic. |
topic_name | The topic name. |
partition_states | The state of each partition |
partition_index | The partition index. |
leader_epoch | The leader epoch. |
delete_partition | Whether this partition should be deleted. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
StopReplica Request (Version: 4) => controller_id is_kraft_controller controller_epoch broker_epoch [topic_states] TAG_BUFFER
controller_id => INT32
is_kraft_controller => BOOLEAN
controller_epoch => INT32
broker_epoch => INT64
topic_states => topic_name [partition_states] TAG_BUFFER
topic_name => COMPACT_STRING
partition_states => partition_index leader_epoch delete_partition TAG_BUFFER
partition_index => INT32
leader_epoch => INT32
delete_partition => BOOLEAN
Field |
Description |
---|
controller_id | The controller id. |
is_kraft_controller | If KRaft controller id is used during migration. See KIP-866 |
controller_epoch | The controller epoch. |
broker_epoch | The broker epoch. |
topic_states | Each topic. |
topic_name | The topic name. |
partition_states | The state of each partition |
partition_index | The partition index. |
leader_epoch | The leader epoch. |
delete_partition | Whether this partition should be deleted. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
StopReplica Response (Version: 0) => error_code [partition_errors]
error_code => INT16
partition_errors => topic_name partition_index error_code
topic_name => STRING
partition_index => INT32
error_code => INT16
Field |
Description |
---|
error_code | The top-level error code, or 0 if there was no top-level error. |
partition_errors | The responses for each partition. |
topic_name | The topic name. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no partition error. |
StopReplica Response (Version: 1) => error_code [partition_errors]
error_code => INT16
partition_errors => topic_name partition_index error_code
topic_name => STRING
partition_index => INT32
error_code => INT16
Field |
Description |
---|
error_code | The top-level error code, or 0 if there was no top-level error. |
partition_errors | The responses for each partition. |
topic_name | The topic name. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no partition error. |
StopReplica Response (Version: 2) => error_code [partition_errors] TAG_BUFFER
error_code => INT16
partition_errors => topic_name partition_index error_code TAG_BUFFER
topic_name => COMPACT_STRING
partition_index => INT32
error_code => INT16
Field |
Description |
---|
error_code | The top-level error code, or 0 if there was no top-level error. |
partition_errors | The responses for each partition. |
topic_name | The topic name. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no partition error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
StopReplica Response (Version: 3) => error_code [partition_errors] TAG_BUFFER
error_code => INT16
partition_errors => topic_name partition_index error_code TAG_BUFFER
topic_name => COMPACT_STRING
partition_index => INT32
error_code => INT16
Field |
Description |
---|
error_code | The top-level error code, or 0 if there was no top-level error. |
partition_errors | The responses for each partition. |
topic_name | The topic name. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no partition error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
StopReplica Response (Version: 4) => error_code [partition_errors] TAG_BUFFER
error_code => INT16
partition_errors => topic_name partition_index error_code TAG_BUFFER
topic_name => COMPACT_STRING
partition_index => INT32
error_code => INT16
Field |
Description |
---|
error_code | The top-level error code, or 0 if there was no top-level error. |
partition_errors | The responses for each partition. |
topic_name | The topic name. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no partition error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
UpdateMetadata Request (Version: 0) => controller_id controller_epoch [ungrouped_partition_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas]
topic_name => STRING
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
live_brokers => id v0_host v0_port
id => INT32
v0_host => STRING
v0_port => INT32
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
ungrouped_partition_states | In older versions of this RPC, each partition that we would like to update. |
topic_name | In older versions of this RPC, the topic name. |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The ID of the broker which is the current partition leader. |
leader_epoch | The leader epoch of this partition. |
isr | The brokers which are in the ISR for this partition. |
zk_version | The Zookeeper version. |
replicas | All the replicas of this partition. |
live_brokers | |
id | The broker id. |
v0_host | The broker hostname. |
v0_port | The broker port. |
UpdateMetadata Request (Version: 1) => controller_id controller_epoch [ungrouped_partition_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas]
topic_name => STRING
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
live_brokers => id [endpoints]
id => INT32
endpoints => port host security_protocol
port => INT32
host => STRING
security_protocol => INT16
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
ungrouped_partition_states | In older versions of this RPC, each partition that we would like to update. |
topic_name | In older versions of this RPC, the topic name. |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The ID of the broker which is the current partition leader. |
leader_epoch | The leader epoch of this partition. |
isr | The brokers which are in the ISR for this partition. |
zk_version | The Zookeeper version. |
replicas | All the replicas of this partition. |
live_brokers | |
id | The broker id. |
endpoints | The broker endpoints. |
port | The port of this endpoint |
host | The hostname of this endpoint |
security_protocol | The security protocol type. |
UpdateMetadata Request (Version: 2) => controller_id controller_epoch [ungrouped_partition_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas]
topic_name => STRING
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
live_brokers => id [endpoints] rack
id => INT32
endpoints => port host security_protocol
port => INT32
host => STRING
security_protocol => INT16
rack => NULLABLE_STRING
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
ungrouped_partition_states | In older versions of this RPC, each partition that we would like to update. |
topic_name | In older versions of this RPC, the topic name. |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The ID of the broker which is the current partition leader. |
leader_epoch | The leader epoch of this partition. |
isr | The brokers which are in the ISR for this partition. |
zk_version | The Zookeeper version. |
replicas | All the replicas of this partition. |
live_brokers | |
id | The broker id. |
endpoints | The broker endpoints. |
port | The port of this endpoint |
host | The hostname of this endpoint |
security_protocol | The security protocol type. |
rack | The rack which this broker belongs to. |
UpdateMetadata Request (Version: 3) => controller_id controller_epoch [ungrouped_partition_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas]
topic_name => STRING
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
live_brokers => id [endpoints] rack
id => INT32
endpoints => port host listener security_protocol
port => INT32
host => STRING
listener => STRING
security_protocol => INT16
rack => NULLABLE_STRING
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
ungrouped_partition_states | In older versions of this RPC, each partition that we would like to update. |
topic_name | In older versions of this RPC, the topic name. |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The ID of the broker which is the current partition leader. |
leader_epoch | The leader epoch of this partition. |
isr | The brokers which are in the ISR for this partition. |
zk_version | The Zookeeper version. |
replicas | All the replicas of this partition. |
live_brokers | |
id | The broker id. |
endpoints | The broker endpoints. |
port | The port of this endpoint |
host | The hostname of this endpoint |
listener | The listener name. |
security_protocol | The security protocol type. |
rack | The rack which this broker belongs to. |
UpdateMetadata Request (Version: 4) => controller_id controller_epoch [ungrouped_partition_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] [offline_replicas]
topic_name => STRING
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
offline_replicas => INT32
live_brokers => id [endpoints] rack
id => INT32
endpoints => port host listener security_protocol
port => INT32
host => STRING
listener => STRING
security_protocol => INT16
rack => NULLABLE_STRING
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
ungrouped_partition_states | In older versions of this RPC, each partition that we would like to update. |
topic_name | In older versions of this RPC, the topic name. |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The ID of the broker which is the current partition leader. |
leader_epoch | The leader epoch of this partition. |
isr | The brokers which are in the ISR for this partition. |
zk_version | The Zookeeper version. |
replicas | All the replicas of this partition. |
offline_replicas | The replicas of this partition which are offline. |
live_brokers | |
id | The broker id. |
endpoints | The broker endpoints. |
port | The port of this endpoint |
host | The hostname of this endpoint |
listener | The listener name. |
security_protocol | The security protocol type. |
rack | The rack which this broker belongs to. |
UpdateMetadata Request (Version: 5) => controller_id controller_epoch broker_epoch [topic_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
topic_states => topic_name [partition_states]
topic_name => STRING
partition_states => partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] [offline_replicas]
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
offline_replicas => INT32
live_brokers => id [endpoints] rack
id => INT32
endpoints => port host listener security_protocol
port => INT32
host => STRING
listener => STRING
security_protocol => INT16
rack => NULLABLE_STRING
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
broker_epoch | The broker epoch. |
topic_states | In newer versions of this RPC, each topic that we would like to update. |
topic_name | The topic name. |
partition_states | The partition that we would like to update. |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The ID of the broker which is the current partition leader. |
leader_epoch | The leader epoch of this partition. |
isr | The brokers which are in the ISR for this partition. |
zk_version | The Zookeeper version. |
replicas | All the replicas of this partition. |
offline_replicas | The replicas of this partition which are offline. |
live_brokers | |
id | The broker id. |
endpoints | The broker endpoints. |
port | The port of this endpoint |
host | The hostname of this endpoint |
listener | The listener name. |
security_protocol | The security protocol type. |
rack | The rack which this broker belongs to. |
UpdateMetadata Request (Version: 6) => controller_id controller_epoch broker_epoch [topic_states] [live_brokers] TAG_BUFFER
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
topic_states => topic_name [partition_states] TAG_BUFFER
topic_name => COMPACT_STRING
partition_states => partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] [offline_replicas] TAG_BUFFER
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
offline_replicas => INT32
live_brokers => id [endpoints] rack TAG_BUFFER
id => INT32
endpoints => port host listener security_protocol TAG_BUFFER
port => INT32
host => COMPACT_STRING
listener => COMPACT_STRING
security_protocol => INT16
rack => COMPACT_NULLABLE_STRING
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
broker_epoch | The broker epoch. |
topic_states | In newer versions of this RPC, each topic that we would like to update. |
topic_name | The topic name. |
partition_states | The partition that we would like to update. |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The ID of the broker which is the current partition leader. |
leader_epoch | The leader epoch of this partition. |
isr | The brokers which are in the ISR for this partition. |
zk_version | The Zookeeper version. |
replicas | All the replicas of this partition. |
offline_replicas | The replicas of this partition which are offline. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
live_brokers | |
id | The broker id. |
endpoints | The broker endpoints. |
port | The port of this endpoint |
host | The hostname of this endpoint |
listener | The listener name. |
security_protocol | The security protocol type. |
_tagged_fields | The tagged fields |
rack | The rack which this broker belongs to. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
UpdateMetadata Request (Version: 7) => controller_id controller_epoch broker_epoch [topic_states] [live_brokers] TAG_BUFFER
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
topic_states => topic_name topic_id [partition_states] TAG_BUFFER
topic_name => COMPACT_STRING
topic_id => UUID
partition_states => partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] [offline_replicas] TAG_BUFFER
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
offline_replicas => INT32
live_brokers => id [endpoints] rack TAG_BUFFER
id => INT32
endpoints => port host listener security_protocol TAG_BUFFER
port => INT32
host => COMPACT_STRING
listener => COMPACT_STRING
security_protocol => INT16
rack => COMPACT_NULLABLE_STRING
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
broker_epoch | The broker epoch. |
topic_states | In newer versions of this RPC, each topic that we would like to update. |
topic_name | The topic name. |
topic_id | The topic id. |
partition_states | The partition that we would like to update. |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The ID of the broker which is the current partition leader. |
leader_epoch | The leader epoch of this partition. |
isr | The brokers which are in the ISR for this partition. |
zk_version | The Zookeeper version. |
replicas | All the replicas of this partition. |
offline_replicas | The replicas of this partition which are offline. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
live_brokers | |
id | The broker id. |
endpoints | The broker endpoints. |
port | The port of this endpoint |
host | The hostname of this endpoint |
listener | The listener name. |
security_protocol | The security protocol type. |
_tagged_fields | The tagged fields |
rack | The rack which this broker belongs to. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
UpdateMetadata Request (Version: 8) => controller_id is_kraft_controller controller_epoch broker_epoch [topic_states] [live_brokers] TAG_BUFFER
controller_id => INT32
is_kraft_controller => BOOLEAN
controller_epoch => INT32
broker_epoch => INT64
topic_states => topic_name topic_id [partition_states] TAG_BUFFER
topic_name => COMPACT_STRING
topic_id => UUID
partition_states => partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] [offline_replicas] TAG_BUFFER
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
offline_replicas => INT32
live_brokers => id [endpoints] rack TAG_BUFFER
id => INT32
endpoints => port host listener security_protocol TAG_BUFFER
port => INT32
host => COMPACT_STRING
listener => COMPACT_STRING
security_protocol => INT16
rack => COMPACT_NULLABLE_STRING
Field |
Description |
---|
controller_id | The controller id. |
is_kraft_controller | If KRaft controller id is used during migration. See KIP-866 |
controller_epoch | The controller epoch. |
broker_epoch | The broker epoch. |
topic_states | In newer versions of this RPC, each topic that we would like to update. |
topic_name | The topic name. |
topic_id | The topic id. |
partition_states | The partition that we would like to update. |
partition_index | The partition index. |
controller_epoch | The controller epoch. |
leader | The ID of the broker which is the current partition leader. |
leader_epoch | The leader epoch of this partition. |
isr | The brokers which are in the ISR for this partition. |
zk_version | The Zookeeper version. |
replicas | All the replicas of this partition. |
offline_replicas | The replicas of this partition which are offline. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
live_brokers | |
id | The broker id. |
endpoints | The broker endpoints. |
port | The port of this endpoint |
host | The hostname of this endpoint |
listener | The listener name. |
security_protocol | The security protocol type. |
_tagged_fields | The tagged fields |
rack | The rack which this broker belongs to. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
UpdateMetadata Response (Version: 0) => error_code
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
UpdateMetadata Response (Version: 1) => error_code
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
UpdateMetadata Response (Version: 2) => error_code
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
UpdateMetadata Response (Version: 3) => error_code
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
UpdateMetadata Response (Version: 4) => error_code
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
UpdateMetadata Response (Version: 5) => error_code
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
UpdateMetadata Response (Version: 6) => error_code TAG_BUFFER
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
UpdateMetadata Response (Version: 7) => error_code TAG_BUFFER
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
UpdateMetadata Response (Version: 8) => error_code TAG_BUFFER
error_code => INT16
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
Requests:
ControlledShutdown Request (Version: 0) => broker_id
broker_id => INT32
Field |
Description |
---|
broker_id | The id of the broker for which controlled shutdown has been requested. |
ControlledShutdown Request (Version: 1) => broker_id
broker_id => INT32
Field |
Description |
---|
broker_id | The id of the broker for which controlled shutdown has been requested. |
ControlledShutdown Request (Version: 2) => broker_id broker_epoch
broker_id => INT32
broker_epoch => INT64
Field |
Description |
---|
broker_id | The id of the broker for which controlled shutdown has been requested. |
broker_epoch | The broker epoch. |
ControlledShutdown Request (Version: 3) => broker_id broker_epoch TAG_BUFFER
broker_id => INT32
broker_epoch => INT64
Field |
Description |
---|
broker_id | The id of the broker for which controlled shutdown has been requested. |
broker_epoch | The broker epoch. |
_tagged_fields | The tagged fields |
Responses:
ControlledShutdown Response (Version: 0) => error_code [remaining_partitions]
error_code => INT16
remaining_partitions => topic_name partition_index
topic_name => STRING
partition_index => INT32
Field |
Description |
---|
error_code | The top-level error code. |
remaining_partitions | The partitions that the broker still leads. |
topic_name | The name of the topic. |
partition_index | The index of the partition. |
ControlledShutdown Response (Version: 1) => error_code [remaining_partitions]
error_code => INT16
remaining_partitions => topic_name partition_index
topic_name => STRING
partition_index => INT32
Field |
Description |
---|
error_code | The top-level error code. |
remaining_partitions | The partitions that the broker still leads. |
topic_name | The name of the topic. |
partition_index | The index of the partition. |
ControlledShutdown Response (Version: 2) => error_code [remaining_partitions]
error_code => INT16
remaining_partitions => topic_name partition_index
topic_name => STRING
partition_index => INT32
Field |
Description |
---|
error_code | The top-level error code. |
remaining_partitions | The partitions that the broker still leads. |
topic_name | The name of the topic. |
partition_index | The index of the partition. |
ControlledShutdown Response (Version: 3) => error_code [remaining_partitions] TAG_BUFFER
error_code => INT16
remaining_partitions => topic_name partition_index TAG_BUFFER
topic_name => COMPACT_STRING
partition_index => INT32
Field |
Description |
---|
error_code | The top-level error code. |
remaining_partitions | The partitions that the broker still leads. |
topic_name | The name of the topic. |
partition_index | The index of the partition. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
OffsetCommit Request (Version: 0) => group_id [topics]
group_id => STRING
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset committed_metadata
partition_index => INT32
committed_offset => INT64
committed_metadata => NULLABLE_STRING
Field |
Description |
---|
group_id | The unique group identifier. |
topics | The topics to commit offsets for. |
name | The topic name. |
partitions | Each partition to commit offsets for. |
partition_index | The partition index. |
committed_offset | The message offset to be committed. |
committed_metadata | Any associated metadata the client wants to keep. |
OffsetCommit Request (Version: 1) => group_id generation_id member_id [topics]
group_id => STRING
generation_id => INT32
member_id => STRING
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset commit_timestamp committed_metadata
partition_index => INT32
committed_offset => INT64
commit_timestamp => INT64
committed_metadata => NULLABLE_STRING
Field |
Description |
---|
group_id | The unique group identifier. |
generation_id | The generation of the group. |
member_id | The member ID assigned by the group coordinator. |
topics | The topics to commit offsets for. |
name | The topic name. |
partitions | Each partition to commit offsets for. |
partition_index | The partition index. |
committed_offset | The message offset to be committed. |
commit_timestamp | The timestamp of the commit. |
committed_metadata | Any associated metadata the client wants to keep. |
OffsetCommit Request (Version: 2) => group_id generation_id member_id retention_time_ms [topics]
group_id => STRING
generation_id => 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
Field |
Description |
---|
group_id | The unique group identifier. |
generation_id | The generation of the group. |
member_id | The member ID assigned by the group coordinator. |
retention_time_ms | The time period in ms to retain the offset. |
topics | The topics to commit offsets for. |
name | The topic name. |
partitions | Each partition to commit offsets for. |
partition_index | The partition index. |
committed_offset | The message offset to be committed. |
committed_metadata | Any associated metadata the client wants to keep. |
OffsetCommit Request (Version: 3) => group_id generation_id member_id retention_time_ms [topics]
group_id => STRING
generation_id => 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
Field |
Description |
---|
group_id | The unique group identifier. |
generation_id | The generation of the group. |
member_id | The member ID assigned by the group coordinator. |
retention_time_ms | The time period in ms to retain the offset. |
topics | The topics to commit offsets for. |
name | The topic name. |
partitions | Each partition to commit offsets for. |
partition_index | The partition index. |
committed_offset | The message offset to be committed. |
committed_metadata | Any associated metadata the client wants to keep. |
OffsetCommit Request (Version: 4) => group_id generation_id member_id retention_time_ms [topics]
group_id => STRING
generation_id => 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
Field |
Description |
---|
group_id | The unique group identifier. |
generation_id | The generation of the group. |
member_id | The member ID assigned by the group coordinator. |
retention_time_ms | The time period in ms to retain the offset. |
topics | The topics to commit offsets for. |
name | The topic name. |
partitions | Each partition to commit offsets for. |
partition_index | The partition index. |
committed_offset | The message offset to be committed. |
committed_metadata | Any associated metadata the client wants to keep. |
OffsetCommit Request (Version: 5) => group_id generation_id member_id [topics]
group_id => STRING
generation_id => INT32
member_id => STRING
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset committed_metadata
partition_index => INT32
committed_offset => INT64
committed_metadata => NULLABLE_STRING
Field |
Description |
---|
group_id | The unique group identifier. |
generation_id | The generation of the group. |
member_id | The member ID assigned by the group coordinator. |
topics | The topics to commit offsets for. |
name | The topic name. |
partitions | Each partition to commit offsets for. |
partition_index | The partition index. |
committed_offset | The message offset to be committed. |
committed_metadata | Any associated metadata the client wants to keep. |
OffsetCommit Request (Version: 6) => group_id generation_id member_id [topics]
group_id => STRING
generation_id => INT32
member_id => STRING
topics => 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
Field |
Description |
---|
group_id | The unique group identifier. |
generation_id | The generation of the group. |
member_id | The member ID assigned by the group coordinator. |
topics | The topics to commit offsets for. |
name | The topic name. |
partitions | Each partition to commit offsets for. |
partition_index | The partition index. |
committed_offset | The message offset to be committed. |
committed_leader_epoch | The leader epoch of this partition. |
committed_metadata | Any associated metadata the client wants to keep. |
OffsetCommit Request (Version: 7) => group_id generation_id member_id group_instance_id [topics]
group_id => STRING
generation_id => 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
Field |
Description |
---|
group_id | The unique group identifier. |
generation_id | The generation of the group. |
member_id | The member ID assigned by the group coordinator. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
topics | The topics to commit offsets for. |
name | The topic name. |
partitions | Each partition to commit offsets for. |
partition_index | The partition index. |
committed_offset | The message offset to be committed. |
committed_leader_epoch | The leader epoch of this partition. |
committed_metadata | Any associated metadata the client wants to keep. |
OffsetCommit Request (Version: 8) => group_id generation_id member_id group_instance_id [topics] TAG_BUFFER
group_id => COMPACT_STRING
generation_id => INT32
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index committed_offset committed_leader_epoch committed_metadata TAG_BUFFER
partition_index => INT32
committed_offset => INT64
committed_leader_epoch => INT32
committed_metadata => COMPACT_NULLABLE_STRING
Field |
Description |
---|
group_id | The unique group identifier. |
generation_id | The generation of the group. |
member_id | The member ID assigned by the group coordinator. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
topics | The topics to commit offsets for. |
name | The topic name. |
partitions | Each partition to commit offsets for. |
partition_index | The partition index. |
committed_offset | The message offset to be committed. |
committed_leader_epoch | The leader epoch of this partition. |
committed_metadata | Any associated metadata the client wants to keep. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
OffsetCommit Response (Version: 0) => [topics]
topics => name [partitions]
name => STRING
partitions => partition_index error_code
partition_index => INT32
error_code => INT16
Field |
Description |
---|
topics | The responses for each topic. |
name | The topic name. |
partitions | The responses for each partition in the topic. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no error. |
OffsetCommit Response (Version: 1) => [topics]
topics => name [partitions]
name => STRING
partitions => partition_index error_code
partition_index => INT32
error_code => INT16
Field |
Description |
---|
topics | The responses for each topic. |
name | The topic name. |
partitions | The responses for each partition in the topic. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no error. |
OffsetCommit Response (Version: 2) => [topics]
topics => name [partitions]
name => STRING
partitions => partition_index error_code
partition_index => INT32
error_code => INT16
Field |
Description |
---|
topics | The responses for each topic. |
name | The topic name. |
partitions | The responses for each partition in the topic. |
partition_index | The partition index. |
error_code | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses for each topic. |
name | The topic name. |
partitions | The responses for each partition in the topic. |
partition_index | The partition index. |
error_code | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses for each topic. |
name | The topic name. |
partitions | The responses for each partition in the topic. |
partition_index | The partition index. |
error_code | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses for each topic. |
name | The topic name. |
partitions | The responses for each partition in the topic. |
partition_index | The partition index. |
error_code | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses for each topic. |
name | The topic name. |
partitions | The responses for each partition in the topic. |
partition_index | The partition index. |
error_code | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses for each topic. |
name | The topic name. |
partitions | The responses for each partition in the topic. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no error. |
OffsetCommit Response (Version: 8) => throttle_time_ms [topics] TAG_BUFFER
throttle_time_ms => INT32
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index error_code TAG_BUFFER
partition_index => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses for each topic. |
name | The topic name. |
partitions | The responses for each partition in the topic. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
OffsetFetch Request (Version: 0) => group_id [topics]
group_id => STRING
topics => name [partition_indexes]
name => STRING
partition_indexes => INT32
Field |
Description |
---|
group_id | The group to fetch offsets for. |
topics | Each topic we would like to fetch offsets for, or null to fetch offsets for all topics. |
name | The topic name. |
partition_indexes | The partition indexes we would like to fetch offsets for. |
OffsetFetch Request (Version: 1) => group_id [topics]
group_id => STRING
topics => name [partition_indexes]
name => STRING
partition_indexes => INT32
Field |
Description |
---|
group_id | The group to fetch offsets for. |
topics | Each topic we would like to fetch offsets for, or null to fetch offsets for all topics. |
name | The topic name. |
partition_indexes | The 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
Field |
Description |
---|
group_id | The group to fetch offsets for. |
topics | Each topic we would like to fetch offsets for, or null to fetch offsets for all topics. |
name | The topic name. |
partition_indexes | The 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
Field |
Description |
---|
group_id | The group to fetch offsets for. |
topics | Each topic we would like to fetch offsets for, or null to fetch offsets for all topics. |
name | The topic name. |
partition_indexes | The 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
Field |
Description |
---|
group_id | The group to fetch offsets for. |
topics | Each topic we would like to fetch offsets for, or null to fetch offsets for all topics. |
name | The topic name. |
partition_indexes | The 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
Field |
Description |
---|
group_id | The group to fetch offsets for. |
topics | Each topic we would like to fetch offsets for, or null to fetch offsets for all topics. |
name | The topic name. |
partition_indexes | The partition indexes we would like to fetch offsets for. |
OffsetFetch Request (Version: 6) => group_id [topics] TAG_BUFFER
group_id => COMPACT_STRING
topics => name [partition_indexes] TAG_BUFFER
name => COMPACT_STRING
partition_indexes => INT32
Field |
Description |
---|
group_id | The group to fetch offsets for. |
topics | Each topic we would like to fetch offsets for, or null to fetch offsets for all topics. |
name | The topic name. |
partition_indexes | The partition indexes we would like to fetch offsets for. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
OffsetFetch Request (Version: 7) => group_id [topics] require_stable TAG_BUFFER
group_id => COMPACT_STRING
topics => name [partition_indexes] TAG_BUFFER
name => COMPACT_STRING
partition_indexes => INT32
require_stable => BOOLEAN
Field |
Description |
---|
group_id | The group to fetch offsets for. |
topics | Each topic we would like to fetch offsets for, or null to fetch offsets for all topics. |
name | The topic name. |
partition_indexes | The partition indexes we would like to fetch offsets for. |
_tagged_fields | The tagged fields |
require_stable | Whether broker should hold on returning unstable offsets but set a retriable error code for the partitions. |
_tagged_fields | The tagged fields |
OffsetFetch Request (Version: 8) => [groups] require_stable TAG_BUFFER
groups => group_id [topics] TAG_BUFFER
group_id => COMPACT_STRING
topics => name [partition_indexes] TAG_BUFFER
name => COMPACT_STRING
partition_indexes => INT32
require_stable => BOOLEAN
Field |
Description |
---|
groups | Each group we would like to fetch offsets for |
group_id | The group ID. |
topics | Each topic we would like to fetch offsets for, or null to fetch offsets for all topics. |
name | The topic name. |
partition_indexes | The partition indexes we would like to fetch offsets for. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
require_stable | Whether broker should hold on returning unstable offsets but set a retriable error code for the partitions. |
_tagged_fields | The tagged fields |
Responses:
OffsetFetch Response (Version: 0) => [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
Field |
Description |
---|
topics | The responses per topic. |
name | The topic name. |
partitions | The responses per partition |
partition_index | The partition index. |
committed_offset | The committed message offset. |
metadata | The partition metadata. |
error_code | The error code, or 0 if there was no error. |
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
Field |
Description |
---|
topics | The responses per topic. |
name | The topic name. |
partitions | The responses per partition |
partition_index | The partition index. |
committed_offset | The committed message offset. |
metadata | The partition metadata. |
error_code | The 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
Field |
Description |
---|
topics | The responses per topic. |
name | The topic name. |
partitions | The responses per partition |
partition_index | The partition index. |
committed_offset | The committed message offset. |
metadata | The partition metadata. |
error_code | The error code, or 0 if there was no error. |
error_code | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses per topic. |
name | The topic name. |
partitions | The responses per partition |
partition_index | The partition index. |
committed_offset | The committed message offset. |
metadata | The partition metadata. |
error_code | The error code, or 0 if there was no error. |
error_code | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses per topic. |
name | The topic name. |
partitions | The responses per partition |
partition_index | The partition index. |
committed_offset | The committed message offset. |
metadata | The partition metadata. |
error_code | The error code, or 0 if there was no error. |
error_code | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses per topic. |
name | The topic name. |
partitions | The responses per partition |
partition_index | The partition index. |
committed_offset | The committed message offset. |
committed_leader_epoch | The leader epoch. |
metadata | The partition metadata. |
error_code | The error code, or 0 if there was no error. |
error_code | The top-level error code, or 0 if there was no error. |
OffsetFetch Response (Version: 6) => throttle_time_ms [topics] error_code TAG_BUFFER
throttle_time_ms => INT32
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index committed_offset committed_leader_epoch metadata error_code TAG_BUFFER
partition_index => INT32
committed_offset => INT64
committed_leader_epoch => INT32
metadata => COMPACT_NULLABLE_STRING
error_code => INT16
error_code => INT16
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses per topic. |
name | The topic name. |
partitions | The responses per partition |
partition_index | The partition index. |
committed_offset | The committed message offset. |
committed_leader_epoch | The leader epoch. |
metadata | The partition metadata. |
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
error_code | The top-level error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
OffsetFetch Response (Version: 7) => throttle_time_ms [topics] error_code TAG_BUFFER
throttle_time_ms => INT32
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index committed_offset committed_leader_epoch metadata error_code TAG_BUFFER
partition_index => INT32
committed_offset => INT64
committed_leader_epoch => INT32
metadata => COMPACT_NULLABLE_STRING
error_code => INT16
error_code => INT16
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses per topic. |
name | The topic name. |
partitions | The responses per partition |
partition_index | The partition index. |
committed_offset | The committed message offset. |
committed_leader_epoch | The leader epoch. |
metadata | The partition metadata. |
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
error_code | The top-level error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
OffsetFetch Response (Version: 8) => throttle_time_ms [groups] TAG_BUFFER
throttle_time_ms => INT32
groups => group_id [topics] error_code TAG_BUFFER
group_id => COMPACT_STRING
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index committed_offset committed_leader_epoch metadata error_code TAG_BUFFER
partition_index => INT32
committed_offset => INT64
committed_leader_epoch => INT32
metadata => COMPACT_NULLABLE_STRING
error_code => INT16
error_code => INT16
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
groups | The responses per group id. |
group_id | The group ID. |
topics | The responses per topic. |
name | The topic name. |
partitions | The responses per partition |
partition_index | The partition index. |
committed_offset | The committed message offset. |
committed_leader_epoch | The leader epoch. |
metadata | The partition metadata. |
error_code | The partition-level error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
error_code | The group-level error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
FindCoordinator Request (Version: 0) => key
key => STRING
Field |
Description |
---|
key | The coordinator key. |
FindCoordinator Request (Version: 1) => key key_type
key => STRING
key_type => INT8
Field |
Description |
---|
key | The coordinator key. |
key_type | The coordinator key type. (Group, transaction, etc.) |
FindCoordinator Request (Version: 2) => key key_type
key => STRING
key_type => INT8
Field |
Description |
---|
key | The coordinator key. |
key_type | The coordinator key type. (Group, transaction, etc.) |
FindCoordinator Request (Version: 3) => key key_type TAG_BUFFER
key => COMPACT_STRING
key_type => INT8
Field |
Description |
---|
key | The coordinator key. |
key_type | The coordinator key type. (Group, transaction, etc.) |
_tagged_fields | The tagged fields |
FindCoordinator Request (Version: 4) => key_type [coordinator_keys] TAG_BUFFER
key_type => INT8
coordinator_keys => COMPACT_STRING
Field |
Description |
---|
key_type | The coordinator key type. (Group, transaction, etc.) |
coordinator_keys | The coordinator keys. |
_tagged_fields | The tagged fields |
Responses:
FindCoordinator Response (Version: 0) => error_code node_id host port
error_code => INT16
node_id => INT32
host => STRING
port => INT32
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
node_id | The node id. |
host | The host name. |
port | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
node_id | The node id. |
host | The host name. |
port | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
node_id | The node id. |
host | The host name. |
port | The port. |
FindCoordinator Response (Version: 3) => throttle_time_ms error_code error_message node_id host port TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
node_id => INT32
host => COMPACT_STRING
port => INT32
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
node_id | The node id. |
host | The host name. |
port | The port. |
_tagged_fields | The tagged fields |
FindCoordinator Response (Version: 4) => throttle_time_ms [coordinators] TAG_BUFFER
throttle_time_ms => INT32
coordinators => key node_id host port error_code error_message TAG_BUFFER
key => COMPACT_STRING
node_id => INT32
host => COMPACT_STRING
port => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
coordinators | Each coordinator result in the response |
key | The coordinator key. |
node_id | The node id. |
host | The host name. |
port | The port. |
error_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
JoinGroup Request (Version: 0) => group_id session_timeout_ms member_id protocol_type [protocols]
group_id => STRING
session_timeout_ms => INT32
member_id => STRING
protocol_type => STRING
protocols => name metadata
name => STRING
metadata => BYTES
Field |
Description |
---|
group_id | The group identifier. |
session_timeout_ms | The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds. |
member_id | The member id assigned by the group coordinator. |
protocol_type | The unique name the for class of protocols implemented by the group we want to join. |
protocols | The list of protocols that the member supports. |
name | The protocol name. |
metadata | The protocol metadata. |
JoinGroup Request (Version: 1) => 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
Field |
Description |
---|
group_id | The group identifier. |
session_timeout_ms | The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds. |
rebalance_timeout_ms | The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group. |
member_id | The member id assigned by the group coordinator. |
protocol_type | The unique name the for class of protocols implemented by the group we want to join. |
protocols | The list of protocols that the member supports. |
name | The protocol name. |
metadata | The protocol metadata. |
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
Field |
Description |
---|
group_id | The group identifier. |
session_timeout_ms | The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds. |
rebalance_timeout_ms | The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group. |
member_id | The member id assigned by the group coordinator. |
protocol_type | The unique name the for class of protocols implemented by the group we want to join. |
protocols | The list of protocols that the member supports. |
name | The protocol name. |
metadata | The 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
Field |
Description |
---|
group_id | The group identifier. |
session_timeout_ms | The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds. |
rebalance_timeout_ms | The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group. |
member_id | The member id assigned by the group coordinator. |
protocol_type | The unique name the for class of protocols implemented by the group we want to join. |
protocols | The list of protocols that the member supports. |
name | The protocol name. |
metadata | The 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
Field |
Description |
---|
group_id | The group identifier. |
session_timeout_ms | The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds. |
rebalance_timeout_ms | The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group. |
member_id | The member id assigned by the group coordinator. |
protocol_type | The unique name the for class of protocols implemented by the group we want to join. |
protocols | The list of protocols that the member supports. |
name | The protocol name. |
metadata | The 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
Field |
Description |
---|
group_id | The group identifier. |
session_timeout_ms | The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds. |
rebalance_timeout_ms | The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group. |
member_id | The member id assigned by the group coordinator. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
protocol_type | The unique name the for class of protocols implemented by the group we want to join. |
protocols | The list of protocols that the member supports. |
name | The protocol name. |
metadata | The protocol metadata. |
JoinGroup Request (Version: 6) => group_id session_timeout_ms rebalance_timeout_ms member_id group_instance_id protocol_type [protocols] TAG_BUFFER
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 TAG_BUFFER
name => COMPACT_STRING
metadata => COMPACT_BYTES
Field |
Description |
---|
group_id | The group identifier. |
session_timeout_ms | The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds. |
rebalance_timeout_ms | The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group. |
member_id | The member id assigned by the group coordinator. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
protocol_type | The unique name the for class of protocols implemented by the group we want to join. |
protocols | The list of protocols that the member supports. |
name | The protocol name. |
metadata | The protocol metadata. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
JoinGroup Request (Version: 7) => group_id session_timeout_ms rebalance_timeout_ms member_id group_instance_id protocol_type [protocols] TAG_BUFFER
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 TAG_BUFFER
name => COMPACT_STRING
metadata => COMPACT_BYTES
Field |
Description |
---|
group_id | The group identifier. |
session_timeout_ms | The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds. |
rebalance_timeout_ms | The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group. |
member_id | The member id assigned by the group coordinator. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
protocol_type | The unique name the for class of protocols implemented by the group we want to join. |
protocols | The list of protocols that the member supports. |
name | The protocol name. |
metadata | The protocol metadata. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
JoinGroup Request (Version: 8) => group_id session_timeout_ms rebalance_timeout_ms member_id group_instance_id protocol_type [protocols] reason TAG_BUFFER
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 TAG_BUFFER
name => COMPACT_STRING
metadata => COMPACT_BYTES
reason => COMPACT_NULLABLE_STRING
Field |
Description |
---|
group_id | The group identifier. |
session_timeout_ms | The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds. |
rebalance_timeout_ms | The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group. |
member_id | The member id assigned by the group coordinator. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
protocol_type | The unique name the for class of protocols implemented by the group we want to join. |
protocols | The list of protocols that the member supports. |
name | The protocol name. |
metadata | The protocol metadata. |
_tagged_fields | The tagged fields |
reason | The reason why the member (re-)joins the group. |
_tagged_fields | The tagged fields |
JoinGroup Request (Version: 9) => group_id session_timeout_ms rebalance_timeout_ms member_id group_instance_id protocol_type [protocols] reason TAG_BUFFER
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 TAG_BUFFER
name => COMPACT_STRING
metadata => COMPACT_BYTES
reason => COMPACT_NULLABLE_STRING
Field |
Description |
---|
group_id | The group identifier. |
session_timeout_ms | The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds. |
rebalance_timeout_ms | The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group. |
member_id | The member id assigned by the group coordinator. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
protocol_type | The unique name the for class of protocols implemented by the group we want to join. |
protocols | The list of protocols that the member supports. |
name | The protocol name. |
metadata | The protocol metadata. |
_tagged_fields | The tagged fields |
reason | The reason why the member (re-)joins the group. |
_tagged_fields | The tagged fields |
Responses:
JoinGroup Response (Version: 0) => error_code generation_id protocol_name leader member_id [members]
error_code => INT16
generation_id => INT32
protocol_name => STRING
leader => STRING
member_id => STRING
members => member_id metadata
member_id => STRING
metadata => BYTES
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
generation_id | The generation ID of the group. |
protocol_name | The group protocol selected by the coordinator. |
leader | The leader of the group. |
member_id | The member ID assigned by the group coordinator. |
members | |
member_id | The group member ID. |
metadata | The group member metadata. |
JoinGroup Response (Version: 1) => error_code generation_id protocol_name leader member_id [members]
error_code => INT16
generation_id => INT32
protocol_name => STRING
leader => STRING
member_id => STRING
members => member_id metadata
member_id => STRING
metadata => BYTES
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
generation_id | The generation ID of the group. |
protocol_name | The group protocol selected by the coordinator. |
leader | The leader of the group. |
member_id | The member ID assigned by the group coordinator. |
members | |
member_id | The group member ID. |
metadata | The group member metadata. |
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
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
generation_id | The generation ID of the group. |
protocol_name | The group protocol selected by the coordinator. |
leader | The leader of the group. |
member_id | The member ID assigned by the group coordinator. |
members | |
member_id | The group member ID. |
metadata | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
generation_id | The generation ID of the group. |
protocol_name | The group protocol selected by the coordinator. |
leader | The leader of the group. |
member_id | The member ID assigned by the group coordinator. |
members | |
member_id | The group member ID. |
metadata | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
generation_id | The generation ID of the group. |
protocol_name | The group protocol selected by the coordinator. |
leader | The leader of the group. |
member_id | The member ID assigned by the group coordinator. |
members | |
member_id | The group member ID. |
metadata | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
generation_id | The generation ID of the group. |
protocol_name | The group protocol selected by the coordinator. |
leader | The leader of the group. |
member_id | The member ID assigned by the group coordinator. |
members | |
member_id | The group member ID. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
metadata | The group member metadata. |
JoinGroup Response (Version: 6) => throttle_time_ms error_code generation_id protocol_name leader member_id [members] TAG_BUFFER
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 TAG_BUFFER
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
metadata => COMPACT_BYTES
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
generation_id | The generation ID of the group. |
protocol_name | The group protocol selected by the coordinator. |
leader | The leader of the group. |
member_id | The member ID assigned by the group coordinator. |
members | |
member_id | The group member ID. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
metadata | The group member metadata. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
JoinGroup Response (Version: 7) => throttle_time_ms error_code generation_id protocol_type protocol_name leader member_id [members] TAG_BUFFER
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 TAG_BUFFER
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
metadata => COMPACT_BYTES
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
generation_id | The generation ID of the group. |
protocol_type | The group protocol name. |
protocol_name | The group protocol selected by the coordinator. |
leader | The leader of the group. |
member_id | The member ID assigned by the group coordinator. |
members | |
member_id | The group member ID. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
metadata | The group member metadata. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
JoinGroup Response (Version: 8) => throttle_time_ms error_code generation_id protocol_type protocol_name leader member_id [members] TAG_BUFFER
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 TAG_BUFFER
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
metadata => COMPACT_BYTES
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
generation_id | The generation ID of the group. |
protocol_type | The group protocol name. |
protocol_name | The group protocol selected by the coordinator. |
leader | The leader of the group. |
member_id | The member ID assigned by the group coordinator. |
members | |
member_id | The group member ID. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
metadata | The group member metadata. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
JoinGroup Response (Version: 9) => throttle_time_ms error_code generation_id protocol_type protocol_name leader skip_assignment member_id [members] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
generation_id => INT32
protocol_type => COMPACT_NULLABLE_STRING
protocol_name => COMPACT_NULLABLE_STRING
leader => COMPACT_STRING
skip_assignment => BOOLEAN
member_id => COMPACT_STRING
members => member_id group_instance_id metadata TAG_BUFFER
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
metadata => COMPACT_BYTES
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
generation_id | The generation ID of the group. |
protocol_type | The group protocol name. |
protocol_name | The group protocol selected by the coordinator. |
leader | The leader of the group. |
skip_assignment | True if the leader must skip running the assignment. |
member_id | The member ID assigned by the group coordinator. |
members | |
member_id | The group member ID. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
metadata | The group member metadata. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
Heartbeat Request (Version: 0) => group_id generation_id member_id
group_id => STRING
generation_id => INT32
member_id => STRING
Field |
Description |
---|
group_id | The group id. |
generation_id | The generation of the group. |
member_id | The member ID. |
Heartbeat Request (Version: 1) => group_id generation_id member_id
group_id => STRING
generation_id => INT32
member_id => STRING
Field |
Description |
---|
group_id | The group id. |
generation_id | The generation of the group. |
member_id | The member ID. |
Heartbeat Request (Version: 2) => group_id generation_id member_id
group_id => STRING
generation_id => INT32
member_id => STRING
Field |
Description |
---|
group_id | The group id. |
generation_id | The generation of the group. |
member_id | The 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
Field |
Description |
---|
group_id | The group id. |
generation_id | The generation of the group. |
member_id | The member ID. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
Heartbeat Request (Version: 4) => group_id generation_id member_id group_instance_id TAG_BUFFER
group_id => COMPACT_STRING
generation_id => INT32
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
Field |
Description |
---|
group_id | The group id. |
generation_id | The generation of the group. |
member_id | The member ID. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
_tagged_fields | The tagged fields |
Responses:
Heartbeat Response (Version: 0) => error_code
error_code => INT16
Field |
Description |
---|
error_code | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
Heartbeat Response (Version: 4) => throttle_time_ms error_code TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
Requests:
LeaveGroup Request (Version: 0) => group_id member_id
group_id => STRING
member_id => STRING
Field |
Description |
---|
group_id | The ID of the group to leave. |
member_id | The member ID to remove from the group. |
LeaveGroup Request (Version: 1) => group_id member_id
group_id => STRING
member_id => STRING
Field |
Description |
---|
group_id | The ID of the group to leave. |
member_id | The member ID to remove from the group. |
LeaveGroup Request (Version: 2) => group_id member_id
group_id => STRING
member_id => STRING
Field |
Description |
---|
group_id | The ID of the group to leave. |
member_id | The 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
Field |
Description |
---|
group_id | The ID of the group to leave. |
members | List of leaving member identities. |
member_id | The member ID to remove from the group. |
group_instance_id | The group instance ID to remove from the group. |
LeaveGroup Request (Version: 4) => group_id [members] TAG_BUFFER
group_id => COMPACT_STRING
members => member_id group_instance_id TAG_BUFFER
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
Field |
Description |
---|
group_id | The ID of the group to leave. |
members | List of leaving member identities. |
member_id | The member ID to remove from the group. |
group_instance_id | The group instance ID to remove from the group. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
LeaveGroup Request (Version: 5) => group_id [members] TAG_BUFFER
group_id => COMPACT_STRING
members => member_id group_instance_id reason TAG_BUFFER
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
reason => COMPACT_NULLABLE_STRING
Field |
Description |
---|
group_id | The ID of the group to leave. |
members | List of leaving member identities. |
member_id | The member ID to remove from the group. |
group_instance_id | The group instance ID to remove from the group. |
reason | The reason why the member left the group. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
LeaveGroup Response (Version: 0) => error_code
error_code => INT16
Field |
Description |
---|
error_code | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
members | List of leaving member responses. |
member_id | The member ID to remove from the group. |
group_instance_id | The group instance ID to remove from the group. |
error_code | The error code, or 0 if there was no error. |
LeaveGroup Response (Version: 4) => throttle_time_ms error_code [members] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
members => member_id group_instance_id error_code TAG_BUFFER
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
error_code => INT16
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
members | List of leaving member responses. |
member_id | The member ID to remove from the group. |
group_instance_id | The group instance ID to remove from the group. |
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
LeaveGroup Response (Version: 5) => throttle_time_ms error_code [members] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
members => member_id group_instance_id error_code TAG_BUFFER
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
error_code => INT16
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
members | List of leaving member responses. |
member_id | The member ID to remove from the group. |
group_instance_id | The group instance ID to remove from the group. |
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
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
Field |
Description |
---|
group_id | The unique group identifier. |
generation_id | The generation of the group. |
member_id | The member ID assigned by the group. |
assignments | Each assignment. |
member_id | The ID of the member to assign. |
assignment | The 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
Field |
Description |
---|
group_id | The unique group identifier. |
generation_id | The generation of the group. |
member_id | The member ID assigned by the group. |
assignments | Each assignment. |
member_id | The ID of the member to assign. |
assignment | The 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
Field |
Description |
---|
group_id | The unique group identifier. |
generation_id | The generation of the group. |
member_id | The member ID assigned by the group. |
assignments | Each assignment. |
member_id | The ID of the member to assign. |
assignment | The 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
Field |
Description |
---|
group_id | The unique group identifier. |
generation_id | The generation of the group. |
member_id | The member ID assigned by the group. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
assignments | Each assignment. |
member_id | The ID of the member to assign. |
assignment | The member assignment. |
SyncGroup Request (Version: 4) => group_id generation_id member_id group_instance_id [assignments] TAG_BUFFER
group_id => COMPACT_STRING
generation_id => INT32
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
assignments => member_id assignment TAG_BUFFER
member_id => COMPACT_STRING
assignment => COMPACT_BYTES
Field |
Description |
---|
group_id | The unique group identifier. |
generation_id | The generation of the group. |
member_id | The member ID assigned by the group. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
assignments | Each assignment. |
member_id | The ID of the member to assign. |
assignment | The member assignment. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
SyncGroup Request (Version: 5) => group_id generation_id member_id group_instance_id protocol_type protocol_name [assignments] TAG_BUFFER
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 TAG_BUFFER
member_id => COMPACT_STRING
assignment => COMPACT_BYTES
Field |
Description |
---|
group_id | The unique group identifier. |
generation_id | The generation of the group. |
member_id | The member ID assigned by the group. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
protocol_type | The group protocol type. |
protocol_name | The group protocol name. |
assignments | Each assignment. |
member_id | The ID of the member to assign. |
assignment | The member assignment. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
SyncGroup Response (Version: 0) => error_code assignment
error_code => INT16
assignment => BYTES
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
assignment | The member assignment. |
SyncGroup Response (Version: 1) => throttle_time_ms error_code assignment
throttle_time_ms => INT32
error_code => INT16
assignment => BYTES
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
assignment | The member assignment. |
SyncGroup Response (Version: 2) => throttle_time_ms error_code assignment
throttle_time_ms => INT32
error_code => INT16
assignment => BYTES
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
assignment | The member assignment. |
SyncGroup Response (Version: 3) => throttle_time_ms error_code assignment
throttle_time_ms => INT32
error_code => INT16
assignment => BYTES
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
assignment | The member assignment. |
SyncGroup Response (Version: 4) => throttle_time_ms error_code assignment TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
assignment => COMPACT_BYTES
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
assignment | The member assignment. |
_tagged_fields | The tagged fields |
SyncGroup Response (Version: 5) => throttle_time_ms error_code protocol_type protocol_name assignment TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
protocol_type => COMPACT_NULLABLE_STRING
protocol_name => COMPACT_NULLABLE_STRING
assignment => COMPACT_BYTES
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
protocol_type | The group protocol type. |
protocol_name | The group protocol name. |
assignment | The member assignment. |
_tagged_fields | The tagged fields |
Requests:
DescribeGroups Request (Version: 0) => [groups]
groups => STRING
Field |
Description |
---|
groups | The names of the groups to describe |
DescribeGroups Request (Version: 1) => [groups]
groups => STRING
Field |
Description |
---|
groups | The names of the groups to describe |
DescribeGroups Request (Version: 2) => [groups]
groups => STRING
Field |
Description |
---|
groups | The names of the groups to describe |
DescribeGroups Request (Version: 3) => [groups] include_authorized_operations
groups => STRING
include_authorized_operations => BOOLEAN
Field |
Description |
---|
groups | The names of the groups to describe |
include_authorized_operations | Whether to include authorized operations. |
DescribeGroups Request (Version: 4) => [groups] include_authorized_operations
groups => STRING
include_authorized_operations => BOOLEAN
Field |
Description |
---|
groups | The names of the groups to describe |
include_authorized_operations | Whether to include authorized operations. |
DescribeGroups Request (Version: 5) => [groups] include_authorized_operations TAG_BUFFER
groups => COMPACT_STRING
include_authorized_operations => BOOLEAN
Field |
Description |
---|
groups | The names of the groups to describe |
include_authorized_operations | Whether to include authorized operations. |
_tagged_fields | The 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
Field |
Description |
---|
groups | Each described group. |
error_code | The describe error, or 0 if there was no error. |
group_id | The group ID string. |
group_state | The group state string, or the empty string. |
protocol_type | The group protocol type, or the empty string. |
protocol_data | The group protocol data, or the empty string. |
members | The group members. |
member_id | The member ID assigned by the group coordinator. |
client_id | The client ID used in the member's latest join group request. |
client_host | The client host. |
member_metadata | The metadata corresponding to the current group protocol in use. |
member_assignment | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
groups | Each described group. |
error_code | The describe error, or 0 if there was no error. |
group_id | The group ID string. |
group_state | The group state string, or the empty string. |
protocol_type | The group protocol type, or the empty string. |
protocol_data | The group protocol data, or the empty string. |
members | The group members. |
member_id | The member ID assigned by the group coordinator. |
client_id | The client ID used in the member's latest join group request. |
client_host | The client host. |
member_metadata | The metadata corresponding to the current group protocol in use. |
member_assignment | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
groups | Each described group. |
error_code | The describe error, or 0 if there was no error. |
group_id | The group ID string. |
group_state | The group state string, or the empty string. |
protocol_type | The group protocol type, or the empty string. |
protocol_data | The group protocol data, or the empty string. |
members | The group members. |
member_id | The member ID assigned by the group coordinator. |
client_id | The client ID used in the member's latest join group request. |
client_host | The client host. |
member_metadata | The metadata corresponding to the current group protocol in use. |
member_assignment | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
groups | Each described group. |
error_code | The describe error, or 0 if there was no error. |
group_id | The group ID string. |
group_state | The group state string, or the empty string. |
protocol_type | The group protocol type, or the empty string. |
protocol_data | The group protocol data, or the empty string. |
members | The group members. |
member_id | The member ID assigned by the group coordinator. |
client_id | The client ID used in the member's latest join group request. |
client_host | The client host. |
member_metadata | The metadata corresponding to the current group protocol in use. |
member_assignment | The current assignment provided by the group leader. |
authorized_operations | 32-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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
groups | Each described group. |
error_code | The describe error, or 0 if there was no error. |
group_id | The group ID string. |
group_state | The group state string, or the empty string. |
protocol_type | The group protocol type, or the empty string. |
protocol_data | The group protocol data, or the empty string. |
members | The group members. |
member_id | The member ID assigned by the group coordinator. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
client_id | The client ID used in the member's latest join group request. |
client_host | The client host. |
member_metadata | The metadata corresponding to the current group protocol in use. |
member_assignment | The current assignment provided by the group leader. |
authorized_operations | 32-bit bitfield to represent authorized operations for this group. |
DescribeGroups Response (Version: 5) => throttle_time_ms [groups] TAG_BUFFER
throttle_time_ms => INT32
groups => error_code group_id group_state protocol_type protocol_data [members] authorized_operations TAG_BUFFER
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 TAG_BUFFER
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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
groups | Each described group. |
error_code | The describe error, or 0 if there was no error. |
group_id | The group ID string. |
group_state | The group state string, or the empty string. |
protocol_type | The group protocol type, or the empty string. |
protocol_data | The group protocol data, or the empty string. |
members | The group members. |
member_id | The member ID assigned by the group coordinator. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
client_id | The client ID used in the member's latest join group request. |
client_host | The client host. |
member_metadata | The metadata corresponding to the current group protocol in use. |
member_assignment | The current assignment provided by the group leader. |
_tagged_fields | The tagged fields |
authorized_operations | 32-bit bitfield to represent authorized operations for this group. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
ListGroups Request (Version: 0) =>
ListGroups Request (Version: 1) =>
ListGroups Request (Version: 2) =>
ListGroups Request (Version: 3) => TAG_BUFFER
Field |
Description |
---|
_tagged_fields | The tagged fields |
ListGroups Request (Version: 4) => [states_filter] TAG_BUFFER
states_filter => COMPACT_STRING
Field |
Description |
---|
states_filter | The states of the groups we want to list. If empty all groups are returned with their state. |
_tagged_fields | The tagged fields |
Responses:
ListGroups Response (Version: 0) => error_code [groups]
error_code => INT16
groups => group_id protocol_type
group_id => STRING
protocol_type => STRING
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
groups | Each group in the response. |
group_id | The group ID. |
protocol_type | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
groups | Each group in the response. |
group_id | The group ID. |
protocol_type | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
groups | Each group in the response. |
group_id | The group ID. |
protocol_type | The group protocol type. |
ListGroups Response (Version: 3) => throttle_time_ms error_code [groups] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
groups => group_id protocol_type TAG_BUFFER
group_id => COMPACT_STRING
protocol_type => COMPACT_STRING
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
groups | Each group in the response. |
group_id | The group ID. |
protocol_type | The group protocol type. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
ListGroups Response (Version: 4) => throttle_time_ms error_code [groups] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
groups => group_id protocol_type group_state TAG_BUFFER
group_id => COMPACT_STRING
protocol_type => COMPACT_STRING
group_state => COMPACT_STRING
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
groups | Each group in the response. |
group_id | The group ID. |
protocol_type | The group protocol type. |
group_state | The group state name. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
SaslHandshake Request (Version: 0) => mechanism
mechanism => STRING
Field |
Description |
---|
mechanism | The SASL mechanism chosen by the client. |
SaslHandshake Request (Version: 1) => mechanism
mechanism => STRING
Field |
Description |
---|
mechanism | The SASL mechanism chosen by the client. |
Responses:
SaslHandshake Response (Version: 0) => error_code [mechanisms]
error_code => INT16
mechanisms => STRING
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
mechanisms | The mechanisms enabled in the server. |
SaslHandshake Response (Version: 1) => error_code [mechanisms]
error_code => INT16
mechanisms => STRING
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
mechanisms | The mechanisms enabled in the server. |
Requests:
ApiVersions Request (Version: 0) =>
ApiVersions Request (Version: 1) =>
ApiVersions Request (Version: 2) =>
ApiVersions Request (Version: 3) => client_software_name client_software_version TAG_BUFFER
client_software_name => COMPACT_STRING
client_software_version => COMPACT_STRING
Field |
Description |
---|
client_software_name | The name of the client. |
client_software_version | The version of the client. |
_tagged_fields | The 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
Field |
Description |
---|
error_code | The top-level error code. |
api_keys | The APIs supported by the broker. |
api_key | The API index. |
min_version | The minimum supported version, inclusive. |
max_version | The 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
Field |
Description |
---|
error_code | The top-level error code. |
api_keys | The APIs supported by the broker. |
api_key | The API index. |
min_version | The minimum supported version, inclusive. |
max_version | The maximum supported version, inclusive. |
throttle_time_ms | The 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
Field |
Description |
---|
error_code | The top-level error code. |
api_keys | The APIs supported by the broker. |
api_key | The API index. |
min_version | The minimum supported version, inclusive. |
max_version | The maximum supported version, inclusive. |
throttle_time_ms | The 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 TAG_BUFFER
error_code => INT16
api_keys => api_key min_version max_version TAG_BUFFER
api_key => INT16
min_version => INT16
max_version => INT16
throttle_time_ms => INT32
Field |
Description |
---|
error_code | The top-level error code. |
api_keys | The APIs supported by the broker. |
api_key | The API index. |
min_version | The minimum supported version, inclusive. |
max_version | The maximum supported version, inclusive. |
_tagged_fields | The tagged fields |
throttle_time_ms | The 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 | The tagged fields |
Requests:
CreateTopics Request (Version: 0) => [topics] timeout_ms
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
Field |
Description |
---|
topics | The topics to create. |
name | The topic name. |
num_partitions | The 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_factor | The 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. |
assignments | The manual partition assignment, or the empty array if we are using automatic assignment. |
partition_index | The partition index. |
broker_ids | The brokers to place the partition on. |
configs | The custom topic configurations to set. |
name | The configuration name. |
value | The configuration value. |
timeout_ms | How long to wait in milliseconds before timing out the request. |
CreateTopics Request (Version: 1) => [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
Field |
Description |
---|
topics | The topics to create. |
name | The topic name. |
num_partitions | The 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_factor | The 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. |
assignments | The manual partition assignment, or the empty array if we are using automatic assignment. |
partition_index | The partition index. |
broker_ids | The brokers to place the partition on. |
configs | The custom topic configurations to set. |
name | The configuration name. |
value | The configuration value. |
timeout_ms | How long to wait in milliseconds before timing out the request. |
validate_only | If true, check that the topics can be created as specified, but don't create anything. |
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
Field |
Description |
---|
topics | The topics to create. |
name | The topic name. |
num_partitions | The 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_factor | The 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. |
assignments | The manual partition assignment, or the empty array if we are using automatic assignment. |
partition_index | The partition index. |
broker_ids | The brokers to place the partition on. |
configs | The custom topic configurations to set. |
name | The configuration name. |
value | The configuration value. |
timeout_ms | How long to wait in milliseconds before timing out the request. |
validate_only | If 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
Field |
Description |
---|
topics | The topics to create. |
name | The topic name. |
num_partitions | The 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_factor | The 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. |
assignments | The manual partition assignment, or the empty array if we are using automatic assignment. |
partition_index | The partition index. |
broker_ids | The brokers to place the partition on. |
configs | The custom topic configurations to set. |
name | The configuration name. |
value | The configuration value. |
timeout_ms | How long to wait in milliseconds before timing out the request. |
validate_only | If 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
Field |
Description |
---|
topics | The topics to create. |
name | The topic name. |
num_partitions | The 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_factor | The 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. |
assignments | The manual partition assignment, or the empty array if we are using automatic assignment. |
partition_index | The partition index. |
broker_ids | The brokers to place the partition on. |
configs | The custom topic configurations to set. |
name | The configuration name. |
value | The configuration value. |
timeout_ms | How long to wait in milliseconds before timing out the request. |
validate_only | If true, check that the topics can be created as specified, but don't create anything. |
CreateTopics Request (Version: 5) => [topics] timeout_ms validate_only TAG_BUFFER
topics => name num_partitions replication_factor [assignments] [configs] TAG_BUFFER
name => COMPACT_STRING
num_partitions => INT32
replication_factor => INT16
assignments => partition_index [broker_ids] TAG_BUFFER
partition_index => INT32
broker_ids => INT32
configs => name value TAG_BUFFER
name => COMPACT_STRING
value => COMPACT_NULLABLE_STRING
timeout_ms => INT32
validate_only => BOOLEAN
Field |
Description |
---|
topics | The topics to create. |
name | The topic name. |
num_partitions | The 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_factor | The 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. |
assignments | The manual partition assignment, or the empty array if we are using automatic assignment. |
partition_index | The partition index. |
broker_ids | The brokers to place the partition on. |
_tagged_fields | The tagged fields |
configs | The custom topic configurations to set. |
name | The configuration name. |
value | The configuration value. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
timeout_ms | How long to wait in milliseconds before timing out the request. |
validate_only | If true, check that the topics can be created as specified, but don't create anything. |
_tagged_fields | The tagged fields |
CreateTopics Request (Version: 6) => [topics] timeout_ms validate_only TAG_BUFFER
topics => name num_partitions replication_factor [assignments] [configs] TAG_BUFFER
name => COMPACT_STRING
num_partitions => INT32
replication_factor => INT16
assignments => partition_index [broker_ids] TAG_BUFFER
partition_index => INT32
broker_ids => INT32
configs => name value TAG_BUFFER
name => COMPACT_STRING
value => COMPACT_NULLABLE_STRING
timeout_ms => INT32
validate_only => BOOLEAN
Field |
Description |
---|
topics | The topics to create. |
name | The topic name. |
num_partitions | The 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_factor | The 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. |
assignments | The manual partition assignment, or the empty array if we are using automatic assignment. |
partition_index | The partition index. |
broker_ids | The brokers to place the partition on. |
_tagged_fields | The tagged fields |
configs | The custom topic configurations to set. |
name | The configuration name. |
value | The configuration value. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
timeout_ms | How long to wait in milliseconds before timing out the request. |
validate_only | If true, check that the topics can be created as specified, but don't create anything. |
_tagged_fields | The tagged fields |
CreateTopics Request (Version: 7) => [topics] timeout_ms validate_only TAG_BUFFER
topics => name num_partitions replication_factor [assignments] [configs] TAG_BUFFER
name => COMPACT_STRING
num_partitions => INT32
replication_factor => INT16
assignments => partition_index [broker_ids] TAG_BUFFER
partition_index => INT32
broker_ids => INT32
configs => name value TAG_BUFFER
name => COMPACT_STRING
value => COMPACT_NULLABLE_STRING
timeout_ms => INT32
validate_only => BOOLEAN
Field |
Description |
---|
topics | The topics to create. |
name | The topic name. |
num_partitions | The 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_factor | The 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. |
assignments | The manual partition assignment, or the empty array if we are using automatic assignment. |
partition_index | The partition index. |
broker_ids | The brokers to place the partition on. |
_tagged_fields | The tagged fields |
configs | The custom topic configurations to set. |
name | The configuration name. |
value | The configuration value. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
timeout_ms | How long to wait in milliseconds before timing out the request. |
validate_only | If true, check that the topics can be created as specified, but don't create anything. |
_tagged_fields | The tagged fields |
Responses:
CreateTopics Response (Version: 0) => [topics]
topics => name error_code
name => STRING
error_code => INT16
Field |
Description |
---|
topics | Results for each topic we tried to create. |
name | The topic name. |
error_code | The error code, or 0 if there was no error. |
CreateTopics Response (Version: 1) => [topics]
topics => name error_code error_message
name => STRING
error_code => INT16
error_message => NULLABLE_STRING
Field |
Description |
---|
topics | Results for each topic we tried to create. |
name | The topic name. |
error_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Results for each topic we tried to create. |
name | The topic name. |
error_code | The error code, or 0 if there was no error. |
error_message | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Results for each topic we tried to create. |
name | The topic name. |
error_code | The error code, or 0 if there was no error. |
error_message | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Results for each topic we tried to create. |
name | The topic name. |
error_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
CreateTopics Response (Version: 5) => throttle_time_ms [topics] TAG_BUFFER
throttle_time_ms => INT32
topics => name error_code error_message num_partitions replication_factor [configs] TAG_BUFFER
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 TAG_BUFFER
name => COMPACT_STRING
value => COMPACT_NULLABLE_STRING
read_only => BOOLEAN
config_source => INT8
is_sensitive => BOOLEAN
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Results for each topic we tried to create. |
name | The topic name. |
error_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
num_partitions | Number of partitions of the topic. |
replication_factor | Replication factor of the topic. |
configs | Configuration of the topic. |
name | The configuration name. |
value | The configuration value. |
read_only | True if the configuration is read-only. |
config_source | The configuration source. |
is_sensitive | True if this configuration is sensitive. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
CreateTopics Response (Version: 6) => throttle_time_ms [topics] TAG_BUFFER
throttle_time_ms => INT32
topics => name error_code error_message num_partitions replication_factor [configs] TAG_BUFFER
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 TAG_BUFFER
name => COMPACT_STRING
value => COMPACT_NULLABLE_STRING
read_only => BOOLEAN
config_source => INT8
is_sensitive => BOOLEAN
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Results for each topic we tried to create. |
name | The topic name. |
error_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
num_partitions | Number of partitions of the topic. |
replication_factor | Replication factor of the topic. |
configs | Configuration of the topic. |
name | The configuration name. |
value | The configuration value. |
read_only | True if the configuration is read-only. |
config_source | The configuration source. |
is_sensitive | True if this configuration is sensitive. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
CreateTopics Response (Version: 7) => throttle_time_ms [topics] TAG_BUFFER
throttle_time_ms => INT32
topics => name topic_id error_code error_message num_partitions replication_factor [configs] TAG_BUFFER
name => COMPACT_STRING
topic_id => UUID
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
num_partitions => INT32
replication_factor => INT16
configs => name value read_only config_source is_sensitive TAG_BUFFER
name => COMPACT_STRING
value => COMPACT_NULLABLE_STRING
read_only => BOOLEAN
config_source => INT8
is_sensitive => BOOLEAN
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Results for each topic we tried to create. |
name | The topic name. |
topic_id | The unique topic ID |
error_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
num_partitions | Number of partitions of the topic. |
replication_factor | Replication factor of the topic. |
configs | Configuration of the topic. |
name | The configuration name. |
value | The configuration value. |
read_only | True if the configuration is read-only. |
config_source | The configuration source. |
is_sensitive | True if this configuration is sensitive. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
DeleteTopics Request (Version: 0) => [topic_names] timeout_ms
topic_names => STRING
timeout_ms => INT32
Field |
Description |
---|
topic_names | The names of the topics to delete |
timeout_ms | The length of time in milliseconds to wait for the deletions to complete. |
DeleteTopics Request (Version: 1) => [topic_names] timeout_ms
topic_names => STRING
timeout_ms => INT32
Field |
Description |
---|
topic_names | The names of the topics to delete |
timeout_ms | The 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
Field |
Description |
---|
topic_names | The names of the topics to delete |
timeout_ms | The 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
Field |
Description |
---|
topic_names | The names of the topics to delete |
timeout_ms | The length of time in milliseconds to wait for the deletions to complete. |
DeleteTopics Request (Version: 4) => [topic_names] timeout_ms TAG_BUFFER
topic_names => COMPACT_STRING
timeout_ms => INT32
Field |
Description |
---|
topic_names | The names of the topics to delete |
timeout_ms | The length of time in milliseconds to wait for the deletions to complete. |
_tagged_fields | The tagged fields |
DeleteTopics Request (Version: 5) => [topic_names] timeout_ms TAG_BUFFER
topic_names => COMPACT_STRING
timeout_ms => INT32
Field |
Description |
---|
topic_names | The names of the topics to delete |
timeout_ms | The length of time in milliseconds to wait for the deletions to complete. |
_tagged_fields | The tagged fields |
DeleteTopics Request (Version: 6) => [topics] timeout_ms TAG_BUFFER
topics => name topic_id TAG_BUFFER
name => COMPACT_NULLABLE_STRING
topic_id => UUID
timeout_ms => INT32
Field |
Description |
---|
topics | The name or topic ID of the topic |
name | The topic name |
topic_id | The unique topic ID |
_tagged_fields | The tagged fields |
timeout_ms | The length of time in milliseconds to wait for the deletions to complete. |
_tagged_fields | The tagged fields |
Responses:
DeleteTopics Response (Version: 0) => [responses]
responses => name error_code
name => STRING
error_code => INT16
Field |
Description |
---|
responses | The results for each topic we tried to delete. |
name | The topic name |
error_code | The deletion error, or 0 if the deletion succeeded. |
DeleteTopics Response (Version: 1) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => name error_code
name => STRING
error_code => INT16
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The results for each topic we tried to delete. |
name | The topic name |
error_code | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The results for each topic we tried to delete. |
name | The topic name |
error_code | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The results for each topic we tried to delete. |
name | The topic name |
error_code | The deletion error, or 0 if the deletion succeeded. |
DeleteTopics Response (Version: 4) => throttle_time_ms [responses] TAG_BUFFER
throttle_time_ms => INT32
responses => name error_code TAG_BUFFER
name => COMPACT_STRING
error_code => INT16
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The results for each topic we tried to delete. |
name | The topic name |
error_code | The deletion error, or 0 if the deletion succeeded. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DeleteTopics Response (Version: 5) => throttle_time_ms [responses] TAG_BUFFER
throttle_time_ms => INT32
responses => name error_code error_message TAG_BUFFER
name => COMPACT_STRING
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The results for each topic we tried to delete. |
name | The topic name |
error_code | The deletion error, or 0 if the deletion succeeded. |
error_message | The error message, or null if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DeleteTopics Response (Version: 6) => throttle_time_ms [responses] TAG_BUFFER
throttle_time_ms => INT32
responses => name topic_id error_code error_message TAG_BUFFER
name => COMPACT_NULLABLE_STRING
topic_id => UUID
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The results for each topic we tried to delete. |
name | The topic name |
topic_id | the unique topic ID |
error_code | The deletion error, or 0 if the deletion succeeded. |
error_message | The error message, or null if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
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
Field |
Description |
---|
topics | Each topic that we want to delete records from. |
name | The topic name. |
partitions | Each partition that we want to delete records from. |
partition_index | The partition index. |
offset | The deletion offset. |
timeout_ms | How 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
Field |
Description |
---|
topics | Each topic that we want to delete records from. |
name | The topic name. |
partitions | Each partition that we want to delete records from. |
partition_index | The partition index. |
offset | The deletion offset. |
timeout_ms | How long to wait for the deletion to complete, in milliseconds. |
DeleteRecords Request (Version: 2) => [topics] timeout_ms TAG_BUFFER
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index offset TAG_BUFFER
partition_index => INT32
offset => INT64
timeout_ms => INT32
Field |
Description |
---|
topics | Each topic that we want to delete records from. |
name | The topic name. |
partitions | Each partition that we want to delete records from. |
partition_index | The partition index. |
offset | The deletion offset. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
timeout_ms | How long to wait for the deletion to complete, in milliseconds. |
_tagged_fields | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Each topic that we wanted to delete records from. |
name | The topic name. |
partitions | Each partition that we wanted to delete records from. |
partition_index | The partition index. |
low_watermark | The partition low water mark. |
error_code | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Each topic that we wanted to delete records from. |
name | The topic name. |
partitions | Each partition that we wanted to delete records from. |
partition_index | The partition index. |
low_watermark | The partition low water mark. |
error_code | The deletion error code, or 0 if the deletion succeeded. |
DeleteRecords Response (Version: 2) => throttle_time_ms [topics] TAG_BUFFER
throttle_time_ms => INT32
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index low_watermark error_code TAG_BUFFER
partition_index => INT32
low_watermark => INT64
error_code => INT16
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Each topic that we wanted to delete records from. |
name | The topic name. |
partitions | Each partition that we wanted to delete records from. |
partition_index | The partition index. |
low_watermark | The partition low water mark. |
error_code | The deletion error code, or 0 if the deletion succeeded. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
InitProducerId Request (Version: 0) => transactional_id transaction_timeout_ms
transactional_id => NULLABLE_STRING
transaction_timeout_ms => INT32
Field |
Description |
---|
transactional_id | The transactional id, or null if the producer is not transactional. |
transaction_timeout_ms | The 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
Field |
Description |
---|
transactional_id | The transactional id, or null if the producer is not transactional. |
transaction_timeout_ms | The 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 TAG_BUFFER
transactional_id => COMPACT_NULLABLE_STRING
transaction_timeout_ms => INT32
Field |
Description |
---|
transactional_id | The transactional id, or null if the producer is not transactional. |
transaction_timeout_ms | The time in ms to wait before aborting idle transactions sent by this producer. This is only relevant if a TransactionalId has been defined. |
_tagged_fields | The tagged fields |
InitProducerId Request (Version: 3) => transactional_id transaction_timeout_ms producer_id producer_epoch TAG_BUFFER
transactional_id => COMPACT_NULLABLE_STRING
transaction_timeout_ms => INT32
producer_id => INT64
producer_epoch => INT16
Field |
Description |
---|
transactional_id | The transactional id, or null if the producer is not transactional. |
transaction_timeout_ms | The time in ms to wait before aborting idle transactions sent by this producer. This is only relevant if a TransactionalId has been defined. |
producer_id | The producer id. This is used to disambiguate requests if a transactional id is reused following its expiration. |
producer_epoch | The 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_fields | The tagged fields |
InitProducerId Request (Version: 4) => transactional_id transaction_timeout_ms producer_id producer_epoch TAG_BUFFER
transactional_id => COMPACT_NULLABLE_STRING
transaction_timeout_ms => INT32
producer_id => INT64
producer_epoch => INT16
Field |
Description |
---|
transactional_id | The transactional id, or null if the producer is not transactional. |
transaction_timeout_ms | The time in ms to wait before aborting idle transactions sent by this producer. This is only relevant if a TransactionalId has been defined. |
producer_id | The producer id. This is used to disambiguate requests if a transactional id is reused following its expiration. |
producer_epoch | The 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_fields | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
producer_id | The current producer id. |
producer_epoch | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
producer_id | The current producer id. |
producer_epoch | The current epoch associated with the producer id. |
InitProducerId Response (Version: 2) => throttle_time_ms error_code producer_id producer_epoch TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
producer_id => INT64
producer_epoch => INT16
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
producer_id | The current producer id. |
producer_epoch | The current epoch associated with the producer id. |
_tagged_fields | The tagged fields |
InitProducerId Response (Version: 3) => throttle_time_ms error_code producer_id producer_epoch TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
producer_id => INT64
producer_epoch => INT16
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
producer_id | The current producer id. |
producer_epoch | The current epoch associated with the producer id. |
_tagged_fields | The tagged fields |
InitProducerId Response (Version: 4) => throttle_time_ms error_code producer_id producer_epoch TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
producer_id => INT64
producer_epoch => INT16
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
producer_id | The current producer id. |
producer_epoch | The current epoch associated with the producer id. |
_tagged_fields | The tagged fields |
Requests:
OffsetForLeaderEpoch Request (Version: 0) => [topics]
topics => topic [partitions]
topic => STRING
partitions => partition leader_epoch
partition => INT32
leader_epoch => INT32
Field |
Description |
---|
topics | Each topic to get offsets for. |
topic | The topic name. |
partitions | Each partition to get offsets for. |
partition | The partition index. |
leader_epoch | The epoch to look up an offset for. |
OffsetForLeaderEpoch Request (Version: 1) => [topics]
topics => topic [partitions]
topic => STRING
partitions => partition leader_epoch
partition => INT32
leader_epoch => INT32
Field |
Description |
---|
topics | Each topic to get offsets for. |
topic | The topic name. |
partitions | Each partition to get offsets for. |
partition | The partition index. |
leader_epoch | The epoch to look up an offset for. |
OffsetForLeaderEpoch Request (Version: 2) => [topics]
topics => topic [partitions]
topic => STRING
partitions => partition current_leader_epoch leader_epoch
partition => INT32
current_leader_epoch => INT32
leader_epoch => INT32
Field |
Description |
---|
topics | Each topic to get offsets for. |
topic | The topic name. |
partitions | Each partition to get offsets for. |
partition | The partition index. |
current_leader_epoch | An 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_epoch | The 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
Field |
Description |
---|
replica_id | The broker ID of the follower, of -1 if this request is from a consumer. |
topics | Each topic to get offsets for. |
topic | The topic name. |
partitions | Each partition to get offsets for. |
partition | The partition index. |
current_leader_epoch | An 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_epoch | The epoch to look up an offset for. |
OffsetForLeaderEpoch Request (Version: 4) => replica_id [topics] TAG_BUFFER
replica_id => INT32
topics => topic [partitions] TAG_BUFFER
topic => COMPACT_STRING
partitions => partition current_leader_epoch leader_epoch TAG_BUFFER
partition => INT32
current_leader_epoch => INT32
leader_epoch => INT32
Field |
Description |
---|
replica_id | The broker ID of the follower, of -1 if this request is from a consumer. |
topics | Each topic to get offsets for. |
topic | The topic name. |
partitions | Each partition to get offsets for. |
partition | The partition index. |
current_leader_epoch | An 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_epoch | The epoch to look up an offset for. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
OffsetForLeaderEpoch Response (Version: 0) => [topics]
topics => topic [partitions]
topic => STRING
partitions => error_code partition end_offset
error_code => INT16
partition => INT32
end_offset => INT64
Field |
Description |
---|
topics | Each topic we fetched offsets for. |
topic | The topic name. |
partitions | Each partition in the topic we fetched offsets for. |
error_code | The error code 0, or if there was no error. |
partition | The partition index. |
end_offset | The end offset of the epoch. |
OffsetForLeaderEpoch Response (Version: 1) => [topics]
topics => topic [partitions]
topic => STRING
partitions => error_code partition leader_epoch end_offset
error_code => INT16
partition => INT32
leader_epoch => INT32
end_offset => INT64
Field |
Description |
---|
topics | Each topic we fetched offsets for. |
topic | The topic name. |
partitions | Each partition in the topic we fetched offsets for. |
error_code | The error code 0, or if there was no error. |
partition | The partition index. |
leader_epoch | The leader epoch of the partition. |
end_offset | The end offset of the epoch. |
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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Each topic we fetched offsets for. |
topic | The topic name. |
partitions | Each partition in the topic we fetched offsets for. |
error_code | The error code 0, or if there was no error. |
partition | The partition index. |
leader_epoch | The leader epoch of the partition. |
end_offset | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Each topic we fetched offsets for. |
topic | The topic name. |
partitions | Each partition in the topic we fetched offsets for. |
error_code | The error code 0, or if there was no error. |
partition | The partition index. |
leader_epoch | The leader epoch of the partition. |
end_offset | The end offset of the epoch. |
OffsetForLeaderEpoch Response (Version: 4) => throttle_time_ms [topics] TAG_BUFFER
throttle_time_ms => INT32
topics => topic [partitions] TAG_BUFFER
topic => COMPACT_STRING
partitions => error_code partition leader_epoch end_offset TAG_BUFFER
error_code => INT16
partition => INT32
leader_epoch => INT32
end_offset => INT64
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Each topic we fetched offsets for. |
topic | The topic name. |
partitions | Each partition in the topic we fetched offsets for. |
error_code | The error code 0, or if there was no error. |
partition | The partition index. |
leader_epoch | The leader epoch of the partition. |
end_offset | The end offset of the epoch. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
AddPartitionsToTxn Request (Version: 0) => transactional_id producer_id producer_epoch [topics]
transactional_id => STRING
producer_id => INT64
producer_epoch => INT16
topics => name [partitions]
name => STRING
partitions => INT32
Field |
Description |
---|
transactional_id | The transactional id corresponding to the transaction. |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
topics | The partitions to add to the transaction. |
name | The name of the topic. |
partitions | The partition indexes to add to the transaction |
AddPartitionsToTxn Request (Version: 1) => transactional_id producer_id producer_epoch [topics]
transactional_id => STRING
producer_id => INT64
producer_epoch => INT16
topics => name [partitions]
name => STRING
partitions => INT32
Field |
Description |
---|
transactional_id | The transactional id corresponding to the transaction. |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
topics | The partitions to add to the transaction. |
name | The name of the topic. |
partitions | The partition indexes to add to the transaction |
AddPartitionsToTxn Request (Version: 2) => transactional_id producer_id producer_epoch [topics]
transactional_id => STRING
producer_id => INT64
producer_epoch => INT16
topics => name [partitions]
name => STRING
partitions => INT32
Field |
Description |
---|
transactional_id | The transactional id corresponding to the transaction. |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
topics | The partitions to add to the transaction. |
name | The name of the topic. |
partitions | The partition indexes to add to the transaction |
AddPartitionsToTxn Request (Version: 3) => transactional_id producer_id producer_epoch [topics] TAG_BUFFER
transactional_id => COMPACT_STRING
producer_id => INT64
producer_epoch => INT16
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => INT32
Field |
Description |
---|
transactional_id | The transactional id corresponding to the transaction. |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
topics | The partitions to add to the transaction. |
name | The name of the topic. |
partitions | The partition indexes to add to the transaction |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
AddPartitionsToTxn Response (Version: 0) => throttle_time_ms [results]
throttle_time_ms => INT32
results => name [results]
name => STRING
results => partition_index error_code
partition_index => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each topic. |
name | The topic name. |
results | The results for each partition |
partition_index | The partition indexes. |
error_code | The response error code. |
AddPartitionsToTxn Response (Version: 1) => throttle_time_ms [results]
throttle_time_ms => INT32
results => name [results]
name => STRING
results => partition_index error_code
partition_index => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each topic. |
name | The topic name. |
results | The results for each partition |
partition_index | The partition indexes. |
error_code | The response error code. |
AddPartitionsToTxn Response (Version: 2) => throttle_time_ms [results]
throttle_time_ms => INT32
results => name [results]
name => STRING
results => partition_index error_code
partition_index => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each topic. |
name | The topic name. |
results | The results for each partition |
partition_index | The partition indexes. |
error_code | The response error code. |
AddPartitionsToTxn Response (Version: 3) => throttle_time_ms [results] TAG_BUFFER
throttle_time_ms => INT32
results => name [results] TAG_BUFFER
name => COMPACT_STRING
results => partition_index error_code TAG_BUFFER
partition_index => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each topic. |
name | The topic name. |
results | The results for each partition |
partition_index | The partition indexes. |
error_code | The response error code. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
AddOffsetsToTxn Request (Version: 0) => transactional_id producer_id producer_epoch group_id
transactional_id => STRING
producer_id => INT64
producer_epoch => INT16
group_id => STRING
Field |
Description |
---|
transactional_id | The transactional id corresponding to the transaction. |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
group_id | The unique group identifier. |
AddOffsetsToTxn Request (Version: 1) => transactional_id producer_id producer_epoch group_id
transactional_id => STRING
producer_id => INT64
producer_epoch => INT16
group_id => STRING
Field |
Description |
---|
transactional_id | The transactional id corresponding to the transaction. |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
group_id | The 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
Field |
Description |
---|
transactional_id | The transactional id corresponding to the transaction. |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
group_id | The unique group identifier. |
AddOffsetsToTxn Request (Version: 3) => transactional_id producer_id producer_epoch group_id TAG_BUFFER
transactional_id => COMPACT_STRING
producer_id => INT64
producer_epoch => INT16
group_id => COMPACT_STRING
Field |
Description |
---|
transactional_id | The transactional id corresponding to the transaction. |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
group_id | The unique group identifier. |
_tagged_fields | The tagged fields |
Responses:
AddOffsetsToTxn Response (Version: 0) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The 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
Field |
Description |
---|
throttle_time_ms | 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_code | The 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
Field |
Description |
---|
throttle_time_ms | 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_code | The response error code, or 0 if there was no error. |
AddOffsetsToTxn Response (Version: 3) => throttle_time_ms error_code TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | 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_code | The response error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
Requests:
EndTxn Request (Version: 0) => transactional_id producer_id producer_epoch committed
transactional_id => STRING
producer_id => INT64
producer_epoch => INT16
committed => BOOLEAN
Field |
Description |
---|
transactional_id | The ID of the transaction to end. |
producer_id | The producer ID. |
producer_epoch | The current epoch associated with the producer. |
committed | True 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
Field |
Description |
---|
transactional_id | The ID of the transaction to end. |
producer_id | The producer ID. |
producer_epoch | The current epoch associated with the producer. |
committed | True 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
Field |
Description |
---|
transactional_id | The ID of the transaction to end. |
producer_id | The producer ID. |
producer_epoch | The current epoch associated with the producer. |
committed | True if the transaction was committed, false if it was aborted. |
EndTxn Request (Version: 3) => transactional_id producer_id producer_epoch committed TAG_BUFFER
transactional_id => COMPACT_STRING
producer_id => INT64
producer_epoch => INT16
committed => BOOLEAN
Field |
Description |
---|
transactional_id | The ID of the transaction to end. |
producer_id | The producer ID. |
producer_epoch | The current epoch associated with the producer. |
committed | True if the transaction was committed, false if it was aborted. |
_tagged_fields | The tagged fields |
Responses:
EndTxn Response (Version: 0) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | The 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_code | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
EndTxn Response (Version: 3) => throttle_time_ms error_code TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
Requests:
WriteTxnMarkers Request (Version: 0) => [markers]
markers => producer_id producer_epoch transaction_result [topics] coordinator_epoch
producer_id => INT64
producer_epoch => INT16
transaction_result => BOOLEAN
topics => name [partition_indexes]
name => STRING
partition_indexes => INT32
coordinator_epoch => INT32
Field |
Description |
---|
markers | The transaction markers to be written. |
producer_id | The current producer ID. |
producer_epoch | The current epoch associated with the producer ID. |
transaction_result | The result of the transaction to write to the partitions (false = ABORT, true = COMMIT). |
topics | Each topic that we want to write transaction marker(s) for. |
name | The topic name. |
partition_indexes | The indexes of the partitions to write transaction markers for. |
coordinator_epoch | Epoch associated with the transaction state partition hosted by this transaction coordinator |
WriteTxnMarkers Request (Version: 1) => [markers] TAG_BUFFER
markers => producer_id producer_epoch transaction_result [topics] coordinator_epoch TAG_BUFFER
producer_id => INT64
producer_epoch => INT16
transaction_result => BOOLEAN
topics => name [partition_indexes] TAG_BUFFER
name => COMPACT_STRING
partition_indexes => INT32
coordinator_epoch => INT32
Field |
Description |
---|
markers | The transaction markers to be written. |
producer_id | The current producer ID. |
producer_epoch | The current epoch associated with the producer ID. |
transaction_result | The result of the transaction to write to the partitions (false = ABORT, true = COMMIT). |
topics | Each topic that we want to write transaction marker(s) for. |
name | The topic name. |
partition_indexes | The indexes of the partitions to write transaction markers for. |
_tagged_fields | The tagged fields |
coordinator_epoch | Epoch associated with the transaction state partition hosted by this transaction coordinator |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
WriteTxnMarkers Response (Version: 0) => [markers]
markers => producer_id [topics]
producer_id => INT64
topics => name [partitions]
name => STRING
partitions => partition_index error_code
partition_index => INT32
error_code => INT16
Field |
Description |
---|
markers | The results for writing makers. |
producer_id | The current producer ID in use by the transactional ID. |
topics | The results by topic. |
name | The topic name. |
partitions | The results by partition. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no error. |
WriteTxnMarkers Response (Version: 1) => [markers] TAG_BUFFER
markers => producer_id [topics] TAG_BUFFER
producer_id => INT64
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index error_code TAG_BUFFER
partition_index => INT32
error_code => INT16
Field |
Description |
---|
markers | The results for writing makers. |
producer_id | The current producer ID in use by the transactional ID. |
topics | The results by topic. |
name | The topic name. |
partitions | The results by partition. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
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
Field |
Description |
---|
transactional_id | The ID of the transaction. |
group_id | The ID of the group. |
producer_id | The current producer ID in use by the transactional ID. |
producer_epoch | The current epoch associated with the producer ID. |
topics | Each topic that we want to commit offsets for. |
name | The topic name. |
partitions | The partitions inside the topic that we want to committ offsets for. |
partition_index | The index of the partition within the topic. |
committed_offset | The message offset to be committed. |
committed_metadata | Any 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
Field |
Description |
---|
transactional_id | The ID of the transaction. |
group_id | The ID of the group. |
producer_id | The current producer ID in use by the transactional ID. |
producer_epoch | The current epoch associated with the producer ID. |
topics | Each topic that we want to commit offsets for. |
name | The topic name. |
partitions | The partitions inside the topic that we want to committ offsets for. |
partition_index | The index of the partition within the topic. |
committed_offset | The message offset to be committed. |
committed_metadata | Any 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
Field |
Description |
---|
transactional_id | The ID of the transaction. |
group_id | The ID of the group. |
producer_id | The current producer ID in use by the transactional ID. |
producer_epoch | The current epoch associated with the producer ID. |
topics | Each topic that we want to commit offsets for. |
name | The topic name. |
partitions | The partitions inside the topic that we want to committ offsets for. |
partition_index | The index of the partition within the topic. |
committed_offset | The message offset to be committed. |
committed_leader_epoch | The leader epoch of the last consumed record. |
committed_metadata | Any 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] TAG_BUFFER
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] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index committed_offset committed_leader_epoch committed_metadata TAG_BUFFER
partition_index => INT32
committed_offset => INT64
committed_leader_epoch => INT32
committed_metadata => COMPACT_NULLABLE_STRING
Field |
Description |
---|
transactional_id | The ID of the transaction. |
group_id | The ID of the group. |
producer_id | The current producer ID in use by the transactional ID. |
producer_epoch | The current epoch associated with the producer ID. |
generation_id | The generation of the consumer. |
member_id | The member ID assigned by the group coordinator. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
topics | Each topic that we want to commit offsets for. |
name | The topic name. |
partitions | The partitions inside the topic that we want to committ offsets for. |
partition_index | The index of the partition within the topic. |
committed_offset | The message offset to be committed. |
committed_leader_epoch | The leader epoch of the last consumed record. |
committed_metadata | Any associated metadata the client wants to keep. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses for each topic. |
name | The topic name. |
partitions | The responses for each partition in the topic. |
partition_index | The partition index. |
error_code | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses for each topic. |
name | The topic name. |
partitions | The responses for each partition in the topic. |
partition_index | The partition index. |
error_code | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses for each topic. |
name | The topic name. |
partitions | The responses for each partition in the topic. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no error. |
TxnOffsetCommit Response (Version: 3) => throttle_time_ms [topics] TAG_BUFFER
throttle_time_ms => INT32
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index error_code TAG_BUFFER
partition_index => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses for each topic. |
name | The topic name. |
partitions | The responses for each partition in the topic. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
DescribeAcls Request (Version: 0) => resource_type_filter resource_name_filter principal_filter host_filter operation permission_type
resource_type_filter => INT8
resource_name_filter => NULLABLE_STRING
principal_filter => NULLABLE_STRING
host_filter => NULLABLE_STRING
operation => INT8
permission_type => INT8
Field |
Description |
---|
resource_type_filter | The resource type. |
resource_name_filter | The resource name, or null to match any resource name. |
principal_filter | The principal to match, or null to match any principal. |
host_filter | The host to match, or null to match any host. |
operation | The operation to match. |
permission_type | The permission type to match. |
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
Field |
Description |
---|
resource_type_filter | The resource type. |
resource_name_filter | The resource name, or null to match any resource name. |
pattern_type_filter | The resource pattern to match. |
principal_filter | The principal to match, or null to match any principal. |
host_filter | The host to match, or null to match any host. |
operation | The operation to match. |
permission_type | The permission type to match. |
DescribeAcls Request (Version: 2) => resource_type_filter resource_name_filter pattern_type_filter principal_filter host_filter operation permission_type TAG_BUFFER
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
Field |
Description |
---|
resource_type_filter | The resource type. |
resource_name_filter | The resource name, or null to match any resource name. |
pattern_type_filter | The resource pattern to match. |
principal_filter | The principal to match, or null to match any principal. |
host_filter | The host to match, or null to match any host. |
operation | The operation to match. |
permission_type | The permission type to match. |
_tagged_fields | The tagged fields |
DescribeAcls Request (Version: 3) => resource_type_filter resource_name_filter pattern_type_filter principal_filter host_filter operation permission_type TAG_BUFFER
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
Field |
Description |
---|
resource_type_filter | The resource type. |
resource_name_filter | The resource name, or null to match any resource name. |
pattern_type_filter | The resource pattern to match. |
principal_filter | The principal to match, or null to match any principal. |
host_filter | The host to match, or null to match any host. |
operation | The operation to match. |
permission_type | The permission type to match. |
_tagged_fields | The tagged fields |
Responses:
DescribeAcls Response (Version: 0) => throttle_time_ms error_code error_message [resources]
throttle_time_ms => INT32
error_code => INT16
error_message => NULLABLE_STRING
resources => resource_type resource_name [acls]
resource_type => INT8
resource_name => STRING
acls => principal host operation permission_type
principal => STRING
host => STRING
operation => INT8
permission_type => INT8
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
resources | Each Resource that is referenced in an ACL. |
resource_type | The resource type. |
resource_name | The resource name. |
acls | The ACLs. |
principal | The ACL principal. |
host | The ACL host. |
operation | The ACL operation. |
permission_type | The ACL permission type. |
DescribeAcls Response (Version: 1) => throttle_time_ms error_code error_message [resources]
throttle_time_ms => INT32
error_code => INT16
error_message => NULLABLE_STRING
resources => resource_type resource_name 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
resources | Each Resource that is referenced in an ACL. |
resource_type | The resource type. |
resource_name | The resource name. |
pattern_type | The resource pattern type. |
acls | The ACLs. |
principal | The ACL principal. |
host | The ACL host. |
operation | The ACL operation. |
permission_type | The ACL permission type. |
DescribeAcls Response (Version: 2) => throttle_time_ms error_code error_message [resources] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
resources => resource_type resource_name pattern_type [acls] TAG_BUFFER
resource_type => INT8
resource_name => COMPACT_STRING
pattern_type => INT8
acls => principal host operation permission_type TAG_BUFFER
principal => COMPACT_STRING
host => COMPACT_STRING
operation => INT8
permission_type => INT8
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
resources | Each Resource that is referenced in an ACL. |
resource_type | The resource type. |
resource_name | The resource name. |
pattern_type | The resource pattern type. |
acls | The ACLs. |
principal | The ACL principal. |
host | The ACL host. |
operation | The ACL operation. |
permission_type | The ACL permission type. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DescribeAcls Response (Version: 3) => throttle_time_ms error_code error_message [resources] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
resources => resource_type resource_name pattern_type [acls] TAG_BUFFER
resource_type => INT8
resource_name => COMPACT_STRING
pattern_type => INT8
acls => principal host operation permission_type TAG_BUFFER
principal => COMPACT_STRING
host => COMPACT_STRING
operation => INT8
permission_type => INT8
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
resources | Each Resource that is referenced in an ACL. |
resource_type | The resource type. |
resource_name | The resource name. |
pattern_type | The resource pattern type. |
acls | The ACLs. |
principal | The ACL principal. |
host | The ACL host. |
operation | The ACL operation. |
permission_type | The ACL permission type. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
CreateAcls Request (Version: 0) => [creations]
creations => resource_type resource_name principal host operation permission_type
resource_type => INT8
resource_name => STRING
principal => STRING
host => STRING
operation => INT8
permission_type => INT8
Field |
Description |
---|
creations | The ACLs that we want to create. |
resource_type | The type of the resource. |
resource_name | The resource name for the ACL. |
principal | The principal for the ACL. |
host | The host for the ACL. |
operation | The operation type for the ACL (read, write, etc.). |
permission_type | The permission type for the ACL (allow, deny, etc.). |
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
Field |
Description |
---|
creations | The ACLs that we want to create. |
resource_type | The type of the resource. |
resource_name | The resource name for the ACL. |
resource_pattern_type | The pattern type for the ACL. |
principal | The principal for the ACL. |
host | The host for the ACL. |
operation | The operation type for the ACL (read, write, etc.). |
permission_type | The permission type for the ACL (allow, deny, etc.). |
CreateAcls Request (Version: 2) => [creations] TAG_BUFFER
creations => resource_type resource_name resource_pattern_type principal host operation permission_type TAG_BUFFER
resource_type => INT8
resource_name => COMPACT_STRING
resource_pattern_type => INT8
principal => COMPACT_STRING
host => COMPACT_STRING
operation => INT8
permission_type => INT8
Field |
Description |
---|
creations | The ACLs that we want to create. |
resource_type | The type of the resource. |
resource_name | The resource name for the ACL. |
resource_pattern_type | The pattern type for the ACL. |
principal | The principal for the ACL. |
host | The host for the ACL. |
operation | The operation type for the ACL (read, write, etc.). |
permission_type | The permission type for the ACL (allow, deny, etc.). |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
CreateAcls Request (Version: 3) => [creations] TAG_BUFFER
creations => resource_type resource_name resource_pattern_type principal host operation permission_type TAG_BUFFER
resource_type => INT8
resource_name => COMPACT_STRING
resource_pattern_type => INT8
principal => COMPACT_STRING
host => COMPACT_STRING
operation => INT8
permission_type => INT8
Field |
Description |
---|
creations | The ACLs that we want to create. |
resource_type | The type of the resource. |
resource_name | The resource name for the ACL. |
resource_pattern_type | The pattern type for the ACL. |
principal | The principal for the ACL. |
host | The host for the ACL. |
operation | The operation type for the ACL (read, write, etc.). |
permission_type | The permission type for the ACL (allow, deny, etc.). |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
CreateAcls Response (Version: 0) => throttle_time_ms [results]
throttle_time_ms => INT32
results => error_code error_message
error_code => INT16
error_message => NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each ACL creation. |
error_code | The result error, or zero if there was no error. |
error_message | The result message, or null if there was no error. |
CreateAcls Response (Version: 1) => throttle_time_ms [results]
throttle_time_ms => INT32
results => error_code error_message
error_code => INT16
error_message => NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each ACL creation. |
error_code | The result error, or zero if there was no error. |
error_message | The result message, or null if there was no error. |
CreateAcls Response (Version: 2) => throttle_time_ms [results] TAG_BUFFER
throttle_time_ms => INT32
results => error_code error_message TAG_BUFFER
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each ACL creation. |
error_code | The result error, or zero if there was no error. |
error_message | The result message, or null if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
CreateAcls Response (Version: 3) => throttle_time_ms [results] TAG_BUFFER
throttle_time_ms => INT32
results => error_code error_message TAG_BUFFER
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each ACL creation. |
error_code | The result error, or zero if there was no error. |
error_message | The result message, or null if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
DeleteAcls Request (Version: 0) => [filters]
filters => resource_type_filter resource_name_filter principal_filter host_filter operation permission_type
resource_type_filter => INT8
resource_name_filter => NULLABLE_STRING
principal_filter => NULLABLE_STRING
host_filter => NULLABLE_STRING
operation => INT8
permission_type => INT8
Field |
Description |
---|
filters | The filters to use when deleting ACLs. |
resource_type_filter | The resource type. |
resource_name_filter | The resource name. |
principal_filter | The principal filter, or null to accept all principals. |
host_filter | The host filter, or null to accept all hosts. |
operation | The ACL operation. |
permission_type | The permission type. |
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
Field |
Description |
---|
filters | The filters to use when deleting ACLs. |
resource_type_filter | The resource type. |
resource_name_filter | The resource name. |
pattern_type_filter | The pattern type. |
principal_filter | The principal filter, or null to accept all principals. |
host_filter | The host filter, or null to accept all hosts. |
operation | The ACL operation. |
permission_type | The permission type. |
DeleteAcls Request (Version: 2) => [filters] TAG_BUFFER
filters => resource_type_filter resource_name_filter pattern_type_filter principal_filter host_filter operation permission_type TAG_BUFFER
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
Field |
Description |
---|
filters | The filters to use when deleting ACLs. |
resource_type_filter | The resource type. |
resource_name_filter | The resource name. |
pattern_type_filter | The pattern type. |
principal_filter | The principal filter, or null to accept all principals. |
host_filter | The host filter, or null to accept all hosts. |
operation | The ACL operation. |
permission_type | The permission type. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DeleteAcls Request (Version: 3) => [filters] TAG_BUFFER
filters => resource_type_filter resource_name_filter pattern_type_filter principal_filter host_filter operation permission_type TAG_BUFFER
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
Field |
Description |
---|
filters | The filters to use when deleting ACLs. |
resource_type_filter | The resource type. |
resource_name_filter | The resource name. |
pattern_type_filter | The pattern type. |
principal_filter | The principal filter, or null to accept all principals. |
host_filter | The host filter, or null to accept all hosts. |
operation | The ACL operation. |
permission_type | The permission type. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
DeleteAcls Response (Version: 0) => 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 principal host operation permission_type
error_code => INT16
error_message => NULLABLE_STRING
resource_type => INT8
resource_name => STRING
principal => STRING
host => STRING
operation => INT8
permission_type => INT8
Field |
Description |
---|
throttle_time_ms | The 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_results | The results for each filter. |
error_code | The error code, or 0 if the filter succeeded. |
error_message | The error message, or null if the filter succeeded. |
matching_acls | The ACLs which matched this filter. |
error_code | The deletion error code, or 0 if the deletion succeeded. |
error_message | The deletion error message, or null if the deletion succeeded. |
resource_type | The ACL resource type. |
resource_name | The ACL resource name. |
principal | The ACL principal. |
host | The ACL host. |
operation | The ACL operation. |
permission_type | The ACL permission type. |
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
Field |
Description |
---|
throttle_time_ms | The 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_results | The results for each filter. |
error_code | The error code, or 0 if the filter succeeded. |
error_message | The error message, or null if the filter succeeded. |
matching_acls | The ACLs which matched this filter. |
error_code | The deletion error code, or 0 if the deletion succeeded. |
error_message | The deletion error message, or null if the deletion succeeded. |
resource_type | The ACL resource type. |
resource_name | The ACL resource name. |
pattern_type | The ACL resource pattern type. |
principal | The ACL principal. |
host | The ACL host. |
operation | The ACL operation. |
permission_type | The ACL permission type. |
DeleteAcls Response (Version: 2) => throttle_time_ms [filter_results] TAG_BUFFER
throttle_time_ms => INT32
filter_results => error_code error_message [matching_acls] TAG_BUFFER
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 TAG_BUFFER
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
Field |
Description |
---|
throttle_time_ms | The 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_results | The results for each filter. |
error_code | The error code, or 0 if the filter succeeded. |
error_message | The error message, or null if the filter succeeded. |
matching_acls | The ACLs which matched this filter. |
error_code | The deletion error code, or 0 if the deletion succeeded. |
error_message | The deletion error message, or null if the deletion succeeded. |
resource_type | The ACL resource type. |
resource_name | The ACL resource name. |
pattern_type | The ACL resource pattern type. |
principal | The ACL principal. |
host | The ACL host. |
operation | The ACL operation. |
permission_type | The ACL permission type. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DeleteAcls Response (Version: 3) => throttle_time_ms [filter_results] TAG_BUFFER
throttle_time_ms => INT32
filter_results => error_code error_message [matching_acls] TAG_BUFFER
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 TAG_BUFFER
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
Field |
Description |
---|
throttle_time_ms | The 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_results | The results for each filter. |
error_code | The error code, or 0 if the filter succeeded. |
error_message | The error message, or null if the filter succeeded. |
matching_acls | The ACLs which matched this filter. |
error_code | The deletion error code, or 0 if the deletion succeeded. |
error_message | The deletion error message, or null if the deletion succeeded. |
resource_type | The ACL resource type. |
resource_name | The ACL resource name. |
pattern_type | The ACL resource pattern type. |
principal | The ACL principal. |
host | The ACL host. |
operation | The ACL operation. |
permission_type | The ACL permission type. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
DescribeConfigs Request (Version: 0) => [resources]
resources => resource_type resource_name [configuration_keys]
resource_type => INT8
resource_name => STRING
configuration_keys => STRING
Field |
Description |
---|
resources | The resources whose configurations we want to describe. |
resource_type | The resource type. |
resource_name | The resource name. |
configuration_keys | The configuration keys to list, or null to list all configuration keys. |
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
Field |
Description |
---|
resources | The resources whose configurations we want to describe. |
resource_type | The resource type. |
resource_name | The resource name. |
configuration_keys | The configuration keys to list, or null to list all configuration keys. |
include_synonyms | True 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
Field |
Description |
---|
resources | The resources whose configurations we want to describe. |
resource_type | The resource type. |
resource_name | The resource name. |
configuration_keys | The configuration keys to list, or null to list all configuration keys. |
include_synonyms | True 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
Field |
Description |
---|
resources | The resources whose configurations we want to describe. |
resource_type | The resource type. |
resource_name | The resource name. |
configuration_keys | The configuration keys to list, or null to list all configuration keys. |
include_synonyms | True if we should include all synonyms. |
include_documentation | True if we should include configuration documentation. |
DescribeConfigs Request (Version: 4) => [resources] include_synonyms include_documentation TAG_BUFFER
resources => resource_type resource_name [configuration_keys] TAG_BUFFER
resource_type => INT8
resource_name => COMPACT_STRING
configuration_keys => COMPACT_STRING
include_synonyms => BOOLEAN
include_documentation => BOOLEAN
Field |
Description |
---|
resources | The resources whose configurations we want to describe. |
resource_type | The resource type. |
resource_name | The resource name. |
configuration_keys | The configuration keys to list, or null to list all configuration keys. |
_tagged_fields | The tagged fields |
include_synonyms | True if we should include all synonyms. |
include_documentation | True if we should include configuration documentation. |
_tagged_fields | The tagged fields |
Responses:
DescribeConfigs Response (Version: 0) => 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 is_default is_sensitive
name => STRING
value => NULLABLE_STRING
read_only => BOOLEAN
is_default => BOOLEAN
is_sensitive => BOOLEAN
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each resource. |
error_code | The error code, or 0 if we were able to successfully describe the configurations. |
error_message | The error message, or null if we were able to successfully describe the configurations. |
resource_type | The resource type. |
resource_name | The resource name. |
configs | Each listed configuration. |
name | The configuration name. |
value | The configuration value. |
read_only | True if the configuration is read-only. |
is_default | True if the configuration is not set. |
is_sensitive | True if this configuration is sensitive. |
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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each resource. |
error_code | The error code, or 0 if we were able to successfully describe the configurations. |
error_message | The error message, or null if we were able to successfully describe the configurations. |
resource_type | The resource type. |
resource_name | The resource name. |
configs | Each listed configuration. |
name | The configuration name. |
value | The configuration value. |
read_only | True if the configuration is read-only. |
config_source | The configuration source. |
is_sensitive | True if this configuration is sensitive. |
synonyms | The synonyms for this configuration key. |
name | The synonym name. |
value | The synonym value. |
source | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each resource. |
error_code | The error code, or 0 if we were able to successfully describe the configurations. |
error_message | The error message, or null if we were able to successfully describe the configurations. |
resource_type | The resource type. |
resource_name | The resource name. |
configs | Each listed configuration. |
name | The configuration name. |
value | The configuration value. |
read_only | True if the configuration is read-only. |
config_source | The configuration source. |
is_sensitive | True if this configuration is sensitive. |
synonyms | The synonyms for this configuration key. |
name | The synonym name. |
value | The synonym value. |
source | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each resource. |
error_code | The error code, or 0 if we were able to successfully describe the configurations. |
error_message | The error message, or null if we were able to successfully describe the configurations. |
resource_type | The resource type. |
resource_name | The resource name. |
configs | Each listed configuration. |
name | The configuration name. |
value | The configuration value. |
read_only | True if the configuration is read-only. |
config_source | The configuration source. |
is_sensitive | True if this configuration is sensitive. |
synonyms | The synonyms for this configuration key. |
name | The synonym name. |
value | The synonym value. |
source | The synonym source. |
config_type | The configuration data type. Type can be one of the following values - BOOLEAN, STRING, INT, SHORT, LONG, DOUBLE, LIST, CLASS, PASSWORD |
documentation | The configuration documentation. |
DescribeConfigs Response (Version: 4) => throttle_time_ms [results] TAG_BUFFER
throttle_time_ms => INT32
results => error_code error_message resource_type resource_name [configs] TAG_BUFFER
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
resource_type => INT8
resource_name => COMPACT_STRING
configs => name value read_only config_source is_sensitive [synonyms] config_type documentation TAG_BUFFER
name => COMPACT_STRING
value => COMPACT_NULLABLE_STRING
read_only => BOOLEAN
config_source => INT8
is_sensitive => BOOLEAN
synonyms => name value source TAG_BUFFER
name => COMPACT_STRING
value => COMPACT_NULLABLE_STRING
source => INT8
config_type => INT8
documentation => COMPACT_NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each resource. |
error_code | The error code, or 0 if we were able to successfully describe the configurations. |
error_message | The error message, or null if we were able to successfully describe the configurations. |
resource_type | The resource type. |
resource_name | The resource name. |
configs | Each listed configuration. |
name | The configuration name. |
value | The configuration value. |
read_only | True if the configuration is read-only. |
config_source | The configuration source. |
is_sensitive | True if this configuration is sensitive. |
synonyms | The synonyms for this configuration key. |
name | The synonym name. |
value | The synonym value. |
source | The synonym source. |
_tagged_fields | The tagged fields |
config_type | The configuration data type. Type can be one of the following values - BOOLEAN, STRING, INT, SHORT, LONG, DOUBLE, LIST, CLASS, PASSWORD |
documentation | The configuration documentation. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
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
Field |
Description |
---|
resources | The updates for each resource. |
resource_type | The resource type. |
resource_name | The resource name. |
configs | The configurations. |
name | The configuration key name. |
value | The value to set for the configuration key. |
validate_only | True 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
Field |
Description |
---|
resources | The updates for each resource. |
resource_type | The resource type. |
resource_name | The resource name. |
configs | The configurations. |
name | The configuration key name. |
value | The value to set for the configuration key. |
validate_only | True if we should validate the request, but not change the configurations. |
AlterConfigs Request (Version: 2) => [resources] validate_only TAG_BUFFER
resources => resource_type resource_name [configs] TAG_BUFFER
resource_type => INT8
resource_name => COMPACT_STRING
configs => name value TAG_BUFFER
name => COMPACT_STRING
value => COMPACT_NULLABLE_STRING
validate_only => BOOLEAN
Field |
Description |
---|
resources | The updates for each resource. |
resource_type | The resource type. |
resource_name | The resource name. |
configs | The configurations. |
name | The configuration key name. |
value | The value to set for the configuration key. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
validate_only | True if we should validate the request, but not change the configurations. |
_tagged_fields | The 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
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The responses for each resource. |
error_code | The resource error code. |
error_message | The resource error message, or null if there was no error. |
resource_type | The resource type. |
resource_name | The 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
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The responses for each resource. |
error_code | The resource error code. |
error_message | The resource error message, or null if there was no error. |
resource_type | The resource type. |
resource_name | The resource name. |
AlterConfigs Response (Version: 2) => throttle_time_ms [responses] TAG_BUFFER
throttle_time_ms => INT32
responses => error_code error_message resource_type resource_name TAG_BUFFER
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
resource_type => INT8
resource_name => COMPACT_STRING
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The responses for each resource. |
error_code | The resource error code. |
error_message | The resource error message, or null if there was no error. |
resource_type | The resource type. |
resource_name | The resource name. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
AlterReplicaLogDirs Request (Version: 0) => [dirs]
dirs => path [topics]
path => STRING
topics => name [partitions]
name => STRING
partitions => INT32
Field |
Description |
---|
dirs | The alterations to make for each directory. |
path | The absolute directory path. |
topics | The topics to add to the directory. |
name | The topic name. |
partitions | The partition indexes. |
AlterReplicaLogDirs Request (Version: 1) => [dirs]
dirs => path [topics]
path => STRING
topics => name [partitions]
name => STRING
partitions => INT32
Field |
Description |
---|
dirs | The alterations to make for each directory. |
path | The absolute directory path. |
topics | The topics to add to the directory. |
name | The topic name. |
partitions | The partition indexes. |
AlterReplicaLogDirs Request (Version: 2) => [dirs] TAG_BUFFER
dirs => path [topics] TAG_BUFFER
path => COMPACT_STRING
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => INT32
Field |
Description |
---|
dirs | The alterations to make for each directory. |
path | The absolute directory path. |
topics | The topics to add to the directory. |
name | The topic name. |
partitions | The partition indexes. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
AlterReplicaLogDirs Response (Version: 0) => 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
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each topic. |
topic_name | The name of the topic. |
partitions | The results for each partition. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no error. |
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
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each topic. |
topic_name | The name of the topic. |
partitions | The results for each partition. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no error. |
AlterReplicaLogDirs Response (Version: 2) => throttle_time_ms [results] TAG_BUFFER
throttle_time_ms => INT32
results => topic_name [partitions] TAG_BUFFER
topic_name => COMPACT_STRING
partitions => partition_index error_code TAG_BUFFER
partition_index => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each topic. |
topic_name | The name of the topic. |
partitions | The results for each partition. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
DescribeLogDirs Request (Version: 0) => [topics]
topics => topic [partitions]
topic => STRING
partitions => INT32
Field |
Description |
---|
topics | Each topic that we want to describe log directories for, or null for all topics. |
topic | The topic name |
partitions | The partition indexes. |
DescribeLogDirs Request (Version: 1) => [topics]
topics => topic [partitions]
topic => STRING
partitions => INT32
Field |
Description |
---|
topics | Each topic that we want to describe log directories for, or null for all topics. |
topic | The topic name |
partitions | The partition indexes. |
DescribeLogDirs Request (Version: 2) => [topics] TAG_BUFFER
topics => topic [partitions] TAG_BUFFER
topic => COMPACT_STRING
partitions => INT32
Field |
Description |
---|
topics | Each topic that we want to describe log directories for, or null for all topics. |
topic | The topic name |
partitions | The partition indexes. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DescribeLogDirs Request (Version: 3) => [topics] TAG_BUFFER
topics => topic [partitions] TAG_BUFFER
topic => COMPACT_STRING
partitions => INT32
Field |
Description |
---|
topics | Each topic that we want to describe log directories for, or null for all topics. |
topic | The topic name |
partitions | The partition indexes. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DescribeLogDirs Request (Version: 4) => [topics] TAG_BUFFER
topics => topic [partitions] TAG_BUFFER
topic => COMPACT_STRING
partitions => INT32
Field |
Description |
---|
topics | Each topic that we want to describe log directories for, or null for all topics. |
topic | The topic name |
partitions | The partition indexes. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
DescribeLogDirs Response (Version: 0) => 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The log directories. |
error_code | The error code, or 0 if there was no error. |
log_dir | The absolute log directory path. |
topics | Each topic. |
name | The topic name. |
partitions | |
partition_index | The partition index. |
partition_size | The size of the log segments in this partition in bytes. |
offset_lag | The 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_key | True if this log is created by AlterReplicaLogDirsRequest and will replace the current log of the replica in the future. |
DescribeLogDirs Response (Version: 1) => throttle_time_ms [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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The log directories. |
error_code | The error code, or 0 if there was no error. |
log_dir | The absolute log directory path. |
topics | Each topic. |
name | The topic name. |
partitions | |
partition_index | The partition index. |
partition_size | The size of the log segments in this partition in bytes. |
offset_lag | The 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_key | True 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] TAG_BUFFER
throttle_time_ms => INT32
results => error_code log_dir [topics] TAG_BUFFER
error_code => INT16
log_dir => COMPACT_STRING
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index partition_size offset_lag is_future_key TAG_BUFFER
partition_index => INT32
partition_size => INT64
offset_lag => INT64
is_future_key => BOOLEAN
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The log directories. |
error_code | The error code, or 0 if there was no error. |
log_dir | The absolute log directory path. |
topics | Each topic. |
name | The topic name. |
partitions | |
partition_index | The partition index. |
partition_size | The size of the log segments in this partition in bytes. |
offset_lag | The 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_key | True if this log is created by AlterReplicaLogDirsRequest and will replace the current log of the replica in the future. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DescribeLogDirs Response (Version: 3) => throttle_time_ms error_code [results] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
results => error_code log_dir [topics] TAG_BUFFER
error_code => INT16
log_dir => COMPACT_STRING
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index partition_size offset_lag is_future_key TAG_BUFFER
partition_index => INT32
partition_size => INT64
offset_lag => INT64
is_future_key => BOOLEAN
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
results | The log directories. |
error_code | The error code, or 0 if there was no error. |
log_dir | The absolute log directory path. |
topics | Each topic. |
name | The topic name. |
partitions | |
partition_index | The partition index. |
partition_size | The size of the log segments in this partition in bytes. |
offset_lag | The 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_key | True if this log is created by AlterReplicaLogDirsRequest and will replace the current log of the replica in the future. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DescribeLogDirs Response (Version: 4) => throttle_time_ms error_code [results] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
results => error_code log_dir [topics] total_bytes usable_bytes TAG_BUFFER
error_code => INT16
log_dir => COMPACT_STRING
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index partition_size offset_lag is_future_key TAG_BUFFER
partition_index => INT32
partition_size => INT64
offset_lag => INT64
is_future_key => BOOLEAN
total_bytes => INT64
usable_bytes => INT64
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or 0 if there was no error. |
results | The log directories. |
error_code | The error code, or 0 if there was no error. |
log_dir | The absolute log directory path. |
topics | Each topic. |
name | The topic name. |
partitions | |
partition_index | The partition index. |
partition_size | The size of the log segments in this partition in bytes. |
offset_lag | The 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_key | True if this log is created by AlterReplicaLogDirsRequest and will replace the current log of the replica in the future. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
total_bytes | The total size in bytes of the volume the log directory is in. |
usable_bytes | The usable size in bytes of the volume the log directory is in. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
SaslAuthenticate Request (Version: 0) => auth_bytes
auth_bytes => BYTES
Field |
Description |
---|
auth_bytes | The SASL authentication bytes from the client, as defined by the SASL mechanism. |
SaslAuthenticate Request (Version: 1) => auth_bytes
auth_bytes => BYTES
Field |
Description |
---|
auth_bytes | The SASL authentication bytes from the client, as defined by the SASL mechanism. |
SaslAuthenticate Request (Version: 2) => auth_bytes TAG_BUFFER
auth_bytes => COMPACT_BYTES
Field |
Description |
---|
auth_bytes | The SASL authentication bytes from the client, as defined by the SASL mechanism. |
_tagged_fields | The tagged fields |
Responses:
SaslAuthenticate Response (Version: 0) => error_code error_message auth_bytes
error_code => INT16
error_message => NULLABLE_STRING
auth_bytes => BYTES
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
auth_bytes | The 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
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
auth_bytes | The SASL authentication bytes from the server, as defined by the SASL mechanism. |
session_lifetime_ms | The SASL authentication bytes from the server, as defined by the SASL mechanism. |
SaslAuthenticate Response (Version: 2) => error_code error_message auth_bytes session_lifetime_ms TAG_BUFFER
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
auth_bytes => COMPACT_BYTES
session_lifetime_ms => INT64
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
auth_bytes | The SASL authentication bytes from the server, as defined by the SASL mechanism. |
session_lifetime_ms | The SASL authentication bytes from the server, as defined by the SASL mechanism. |
_tagged_fields | The tagged fields |
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
Field |
Description |
---|
topics | Each topic that we want to create new partitions inside. |
name | The topic name. |
count | The new partition count. |
assignments | The new partition assignments. |
broker_ids | The assigned broker IDs. |
timeout_ms | The time in ms to wait for the partitions to be created. |
validate_only | If 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
Field |
Description |
---|
topics | Each topic that we want to create new partitions inside. |
name | The topic name. |
count | The new partition count. |
assignments | The new partition assignments. |
broker_ids | The assigned broker IDs. |
timeout_ms | The time in ms to wait for the partitions to be created. |
validate_only | If true, then validate the request, but don't actually increase the number of partitions. |
CreatePartitions Request (Version: 2) => [topics] timeout_ms validate_only TAG_BUFFER
topics => name count [assignments] TAG_BUFFER
name => COMPACT_STRING
count => INT32
assignments => [broker_ids] TAG_BUFFER
broker_ids => INT32
timeout_ms => INT32
validate_only => BOOLEAN
Field |
Description |
---|
topics | Each topic that we want to create new partitions inside. |
name | The topic name. |
count | The new partition count. |
assignments | The new partition assignments. |
broker_ids | The assigned broker IDs. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
timeout_ms | The time in ms to wait for the partitions to be created. |
validate_only | If true, then validate the request, but don't actually increase the number of partitions. |
_tagged_fields | The tagged fields |
CreatePartitions Request (Version: 3) => [topics] timeout_ms validate_only TAG_BUFFER
topics => name count [assignments] TAG_BUFFER
name => COMPACT_STRING
count => INT32
assignments => [broker_ids] TAG_BUFFER
broker_ids => INT32
timeout_ms => INT32
validate_only => BOOLEAN
Field |
Description |
---|
topics | Each topic that we want to create new partitions inside. |
name | The topic name. |
count | The new partition count. |
assignments | The new partition assignments. |
broker_ids | The assigned broker IDs. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
timeout_ms | The time in ms to wait for the partitions to be created. |
validate_only | If true, then validate the request, but don't actually increase the number of partitions. |
_tagged_fields | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The partition creation results for each topic. |
name | The topic name. |
error_code | The result error, or zero if there was no error. |
error_message | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The partition creation results for each topic. |
name | The topic name. |
error_code | The result error, or zero if there was no error. |
error_message | The result message, or null if there was no error. |
CreatePartitions Response (Version: 2) => throttle_time_ms [results] TAG_BUFFER
throttle_time_ms => INT32
results => name error_code error_message TAG_BUFFER
name => COMPACT_STRING
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The partition creation results for each topic. |
name | The topic name. |
error_code | The result error, or zero if there was no error. |
error_message | The result message, or null if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
CreatePartitions Response (Version: 3) => throttle_time_ms [results] TAG_BUFFER
throttle_time_ms => INT32
results => name error_code error_message TAG_BUFFER
name => COMPACT_STRING
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The partition creation results for each topic. |
name | The topic name. |
error_code | The result error, or zero if there was no error. |
error_message | The result message, or null if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
CreateDelegationToken Request (Version: 0) => [renewers] max_lifetime_ms
renewers => principal_type principal_name
principal_type => STRING
principal_name => STRING
max_lifetime_ms => INT64
Field |
Description |
---|
renewers | A list of those who are allowed to renew this token before it expires. |
principal_type | The type of the Kafka principal. |
principal_name | The name of the Kafka principal. |
max_lifetime_ms | The maximum lifetime of the token in milliseconds, or -1 to use the server side default. |
CreateDelegationToken Request (Version: 1) => [renewers] max_lifetime_ms
renewers => principal_type principal_name
principal_type => STRING
principal_name => STRING
max_lifetime_ms => INT64
Field |
Description |
---|
renewers | A list of those who are allowed to renew this token before it expires. |
principal_type | The type of the Kafka principal. |
principal_name | The name of the Kafka principal. |
max_lifetime_ms | The maximum lifetime of the token in milliseconds, or -1 to use the server side default. |
CreateDelegationToken Request (Version: 2) => [renewers] max_lifetime_ms TAG_BUFFER
renewers => principal_type principal_name TAG_BUFFER
principal_type => COMPACT_STRING
principal_name => COMPACT_STRING
max_lifetime_ms => INT64
Field |
Description |
---|
renewers | A list of those who are allowed to renew this token before it expires. |
principal_type | The type of the Kafka principal. |
principal_name | The name of the Kafka principal. |
_tagged_fields | The tagged fields |
max_lifetime_ms | The maximum lifetime of the token in milliseconds, or -1 to use the server side default. |
_tagged_fields | The tagged fields |
CreateDelegationToken Request (Version: 3) => owner_principal_type owner_principal_name [renewers] max_lifetime_ms TAG_BUFFER
owner_principal_type => COMPACT_NULLABLE_STRING
owner_principal_name => COMPACT_NULLABLE_STRING
renewers => principal_type principal_name TAG_BUFFER
principal_type => COMPACT_STRING
principal_name => COMPACT_STRING
max_lifetime_ms => INT64
Field |
Description |
---|
owner_principal_type | The principal type of the owner of the token. If it's null it defaults to the token request principal. |
owner_principal_name | The principal name of the owner of the token. If it's null it defaults to the token request principal. |
renewers | A list of those who are allowed to renew this token before it expires. |
principal_type | The type of the Kafka principal. |
principal_name | The name of the Kafka principal. |
_tagged_fields | The tagged fields |
max_lifetime_ms | The maximum lifetime of the token in milliseconds, or -1 to use the server side default. |
_tagged_fields | The tagged fields |
Responses:
CreateDelegationToken Response (Version: 0) => 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
Field |
Description |
---|
error_code | The top-level error, or zero if there was no error. |
principal_type | The principal type of the token owner. |
principal_name | The name of the token owner. |
issue_timestamp_ms | When this token was generated. |
expiry_timestamp_ms | When this token expires. |
max_timestamp_ms | The maximum lifetime of this token. |
token_id | The token UUID. |
hmac | HMAC of the delegation token. |
throttle_time_ms | The 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: 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
Field |
Description |
---|
error_code | The top-level error, or zero if there was no error. |
principal_type | The principal type of the token owner. |
principal_name | The name of the token owner. |
issue_timestamp_ms | When this token was generated. |
expiry_timestamp_ms | When this token expires. |
max_timestamp_ms | The maximum lifetime of this token. |
token_id | The token UUID. |
hmac | HMAC of the delegation token. |
throttle_time_ms | The 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 TAG_BUFFER
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
Field |
Description |
---|
error_code | The top-level error, or zero if there was no error. |
principal_type | The principal type of the token owner. |
principal_name | The name of the token owner. |
issue_timestamp_ms | When this token was generated. |
expiry_timestamp_ms | When this token expires. |
max_timestamp_ms | The maximum lifetime of this token. |
token_id | The token UUID. |
hmac | HMAC of the delegation token. |
throttle_time_ms | The 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 | The tagged fields |
CreateDelegationToken Response (Version: 3) => error_code principal_type principal_name token_requester_principal_type token_requester_principal_name issue_timestamp_ms expiry_timestamp_ms max_timestamp_ms token_id hmac throttle_time_ms TAG_BUFFER
error_code => INT16
principal_type => COMPACT_STRING
principal_name => COMPACT_STRING
token_requester_principal_type => COMPACT_STRING
token_requester_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
Field |
Description |
---|
error_code | The top-level error, or zero if there was no error. |
principal_type | The principal type of the token owner. |
principal_name | The name of the token owner. |
token_requester_principal_type | The principal type of the requester of the token. |
token_requester_principal_name | The principal type of the requester of the token. |
issue_timestamp_ms | When this token was generated. |
expiry_timestamp_ms | When this token expires. |
max_timestamp_ms | The maximum lifetime of this token. |
token_id | The token UUID. |
hmac | HMAC of the delegation token. |
throttle_time_ms | The 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 | The tagged fields |
Requests:
RenewDelegationToken Request (Version: 0) => hmac renew_period_ms
hmac => BYTES
renew_period_ms => INT64
Field |
Description |
---|
hmac | The HMAC of the delegation token to be renewed. |
renew_period_ms | The renewal time period in milliseconds. |
RenewDelegationToken Request (Version: 1) => hmac renew_period_ms
hmac => BYTES
renew_period_ms => INT64
Field |
Description |
---|
hmac | The HMAC of the delegation token to be renewed. |
renew_period_ms | The renewal time period in milliseconds. |
RenewDelegationToken Request (Version: 2) => hmac renew_period_ms TAG_BUFFER
hmac => COMPACT_BYTES
renew_period_ms => INT64
Field |
Description |
---|
hmac | The HMAC of the delegation token to be renewed. |
renew_period_ms | The renewal time period in milliseconds. |
_tagged_fields | The tagged fields |
Responses:
RenewDelegationToken Response (Version: 0) => error_code expiry_timestamp_ms throttle_time_ms
error_code => INT16
expiry_timestamp_ms => INT64
throttle_time_ms => INT32
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
expiry_timestamp_ms | The timestamp in milliseconds at which this token expires. |
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
RenewDelegationToken Response (Version: 1) => error_code expiry_timestamp_ms throttle_time_ms
error_code => INT16
expiry_timestamp_ms => INT64
throttle_time_ms => INT32
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
expiry_timestamp_ms | The timestamp in milliseconds at which this token expires. |
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
RenewDelegationToken Response (Version: 2) => error_code expiry_timestamp_ms throttle_time_ms TAG_BUFFER
error_code => INT16
expiry_timestamp_ms => INT64
throttle_time_ms => INT32
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
expiry_timestamp_ms | The timestamp in milliseconds at which this token expires. |
throttle_time_ms | The 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 | The tagged fields |
Requests:
ExpireDelegationToken Request (Version: 0) => hmac expiry_time_period_ms
hmac => BYTES
expiry_time_period_ms => INT64
Field |
Description |
---|
hmac | The HMAC of the delegation token to be expired. |
expiry_time_period_ms | The expiry time period in milliseconds. |
ExpireDelegationToken Request (Version: 1) => hmac expiry_time_period_ms
hmac => BYTES
expiry_time_period_ms => INT64
Field |
Description |
---|
hmac | The HMAC of the delegation token to be expired. |
expiry_time_period_ms | The expiry time period in milliseconds. |
ExpireDelegationToken Request (Version: 2) => hmac expiry_time_period_ms TAG_BUFFER
hmac => COMPACT_BYTES
expiry_time_period_ms => INT64
Field |
Description |
---|
hmac | The HMAC of the delegation token to be expired. |
expiry_time_period_ms | The expiry time period in milliseconds. |
_tagged_fields | The tagged fields |
Responses:
ExpireDelegationToken Response (Version: 0) => error_code expiry_timestamp_ms throttle_time_ms
error_code => INT16
expiry_timestamp_ms => INT64
throttle_time_ms => INT32
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
expiry_timestamp_ms | The timestamp in milliseconds at which this token expires. |
throttle_time_ms | The 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 Response (Version: 1) => error_code expiry_timestamp_ms throttle_time_ms
error_code => INT16
expiry_timestamp_ms => INT64
throttle_time_ms => INT32
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
expiry_timestamp_ms | The timestamp in milliseconds at which this token expires. |
throttle_time_ms | The 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 Response (Version: 2) => error_code expiry_timestamp_ms throttle_time_ms TAG_BUFFER
error_code => INT16
expiry_timestamp_ms => INT64
throttle_time_ms => INT32
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
expiry_timestamp_ms | The timestamp in milliseconds at which this token expires. |
throttle_time_ms | The 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 | The tagged fields |
Requests:
DescribeDelegationToken Request (Version: 0) => [owners]
owners => principal_type principal_name
principal_type => STRING
principal_name => STRING
Field |
Description |
---|
owners | Each owner that we want to describe delegation tokens for, or null to describe all tokens. |
principal_type | The owner principal type. |
principal_name | The owner principal name. |
DescribeDelegationToken Request (Version: 1) => [owners]
owners => principal_type principal_name
principal_type => STRING
principal_name => STRING
Field |
Description |
---|
owners | Each owner that we want to describe delegation tokens for, or null to describe all tokens. |
principal_type | The owner principal type. |
principal_name | The owner principal name. |
DescribeDelegationToken Request (Version: 2) => [owners] TAG_BUFFER
owners => principal_type principal_name TAG_BUFFER
principal_type => COMPACT_STRING
principal_name => COMPACT_STRING
Field |
Description |
---|
owners | Each owner that we want to describe delegation tokens for, or null to describe all tokens. |
principal_type | The owner principal type. |
principal_name | The owner principal name. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DescribeDelegationToken Request (Version: 3) => [owners] TAG_BUFFER
owners => principal_type principal_name TAG_BUFFER
principal_type => COMPACT_STRING
principal_name => COMPACT_STRING
Field |
Description |
---|
owners | Each owner that we want to describe delegation tokens for, or null to describe all tokens. |
principal_type | The owner principal type. |
principal_name | The owner principal name. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
DescribeDelegationToken Response (Version: 0) => 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
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
tokens | The tokens. |
principal_type | The token principal type. |
principal_name | The token principal name. |
issue_timestamp | The token issue timestamp in milliseconds. |
expiry_timestamp | The token expiry timestamp in milliseconds. |
max_timestamp | The token maximum timestamp length in milliseconds. |
token_id | The token ID. |
hmac | The token HMAC. |
renewers | Those who are able to renew this token before it expires. |
principal_type | The renewer principal type |
principal_name | The renewer principal name |
throttle_time_ms | The 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: 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
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
tokens | The tokens. |
principal_type | The token principal type. |
principal_name | The token principal name. |
issue_timestamp | The token issue timestamp in milliseconds. |
expiry_timestamp | The token expiry timestamp in milliseconds. |
max_timestamp | The token maximum timestamp length in milliseconds. |
token_id | The token ID. |
hmac | The token HMAC. |
renewers | Those who are able to renew this token before it expires. |
principal_type | The renewer principal type |
principal_name | The renewer principal name |
throttle_time_ms | The 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 TAG_BUFFER
error_code => INT16
tokens => principal_type principal_name issue_timestamp expiry_timestamp max_timestamp token_id hmac [renewers] TAG_BUFFER
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 TAG_BUFFER
principal_type => COMPACT_STRING
principal_name => COMPACT_STRING
throttle_time_ms => INT32
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
tokens | The tokens. |
principal_type | The token principal type. |
principal_name | The token principal name. |
issue_timestamp | The token issue timestamp in milliseconds. |
expiry_timestamp | The token expiry timestamp in milliseconds. |
max_timestamp | The token maximum timestamp length in milliseconds. |
token_id | The token ID. |
hmac | The token HMAC. |
renewers | Those who are able to renew this token before it expires. |
principal_type | The renewer principal type |
principal_name | The renewer principal name |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
throttle_time_ms | The 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 | The tagged fields |
DescribeDelegationToken Response (Version: 3) => error_code [tokens] throttle_time_ms TAG_BUFFER
error_code => INT16
tokens => principal_type principal_name token_requester_principal_type token_requester_principal_name issue_timestamp expiry_timestamp max_timestamp token_id hmac [renewers] TAG_BUFFER
principal_type => COMPACT_STRING
principal_name => COMPACT_STRING
token_requester_principal_type => COMPACT_STRING
token_requester_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 TAG_BUFFER
principal_type => COMPACT_STRING
principal_name => COMPACT_STRING
throttle_time_ms => INT32
Field |
Description |
---|
error_code | The error code, or 0 if there was no error. |
tokens | The tokens. |
principal_type | The token principal type. |
principal_name | The token principal name. |
token_requester_principal_type | The principal type of the requester of the token. |
token_requester_principal_name | The principal type of the requester of the token. |
issue_timestamp | The token issue timestamp in milliseconds. |
expiry_timestamp | The token expiry timestamp in milliseconds. |
max_timestamp | The token maximum timestamp length in milliseconds. |
token_id | The token ID. |
hmac | The token HMAC. |
renewers | Those who are able to renew this token before it expires. |
principal_type | The renewer principal type |
principal_name | The renewer principal name |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
throttle_time_ms | The 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 | The tagged fields |
Requests:
DeleteGroups Request (Version: 0) => [groups_names]
groups_names => STRING
Field |
Description |
---|
groups_names | The group names to delete. |
DeleteGroups Request (Version: 1) => [groups_names]
groups_names => STRING
Field |
Description |
---|
groups_names | The group names to delete. |
DeleteGroups Request (Version: 2) => [groups_names] TAG_BUFFER
groups_names => COMPACT_STRING
Field |
Description |
---|
groups_names | The group names to delete. |
_tagged_fields | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The deletion results |
group_id | The group id |
error_code | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The deletion results |
group_id | The group id |
error_code | The deletion error, or 0 if the deletion succeeded. |
DeleteGroups Response (Version: 2) => throttle_time_ms [results] TAG_BUFFER
throttle_time_ms => INT32
results => group_id error_code TAG_BUFFER
group_id => COMPACT_STRING
error_code => INT16
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The deletion results |
group_id | The group id |
error_code | The deletion error, or 0 if the deletion succeeded. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
ElectLeaders Request (Version: 0) => [topic_partitions] timeout_ms
topic_partitions => topic [partitions]
topic => STRING
partitions => INT32
timeout_ms => INT32
Field |
Description |
---|
topic_partitions | The topic partitions to elect leaders. |
topic | The name of a topic. |
partitions | The partitions of this topic whose leader should be elected. |
timeout_ms | The 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
Field |
Description |
---|
election_type | Type 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_partitions | The topic partitions to elect leaders. |
topic | The name of a topic. |
partitions | The partitions of this topic whose leader should be elected. |
timeout_ms | The time in ms to wait for the election to complete. |
ElectLeaders Request (Version: 2) => election_type [topic_partitions] timeout_ms TAG_BUFFER
election_type => INT8
topic_partitions => topic [partitions] TAG_BUFFER
topic => COMPACT_STRING
partitions => INT32
timeout_ms => INT32
Field |
Description |
---|
election_type | Type 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_partitions | The topic partitions to elect leaders. |
topic | The name of a topic. |
partitions | The partitions of this topic whose leader should be elected. |
_tagged_fields | The tagged fields |
timeout_ms | The time in ms to wait for the election to complete. |
_tagged_fields | The 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
Field |
Description |
---|
throttle_time_ms | The 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_results | The election results, or an empty array if the requester did not have permission and the request asks for all partitions. |
topic | The topic name |
partition_result | The results for each partition |
partition_id | The partition id |
error_code | The result error, or zero if there was no error. |
error_message | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The top level response error code. |
replica_election_results | The election results, or an empty array if the requester did not have permission and the request asks for all partitions. |
topic | The topic name |
partition_result | The results for each partition |
partition_id | The partition id |
error_code | The result error, or zero if there was no error. |
error_message | The result message, or null if there was no error. |
ElectLeaders Response (Version: 2) => throttle_time_ms error_code [replica_election_results] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
replica_election_results => topic [partition_result] TAG_BUFFER
topic => COMPACT_STRING
partition_result => partition_id error_code error_message TAG_BUFFER
partition_id => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | The 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_code | The top level response error code. |
replica_election_results | The election results, or an empty array if the requester did not have permission and the request asks for all partitions. |
topic | The topic name |
partition_result | The results for each partition |
partition_id | The partition id |
error_code | The result error, or zero if there was no error. |
error_message | The result message, or null if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
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
Field |
Description |
---|
resources | The incremental updates for each resource. |
resource_type | The resource type. |
resource_name | The resource name. |
configs | The configurations. |
name | The configuration key name. |
config_operation | The type (Set, Delete, Append, Subtract) of operation. |
value | The value to set for the configuration key. |
validate_only | True if we should validate the request, but not change the configurations. |
IncrementalAlterConfigs Request (Version: 1) => [resources] validate_only TAG_BUFFER
resources => resource_type resource_name [configs] TAG_BUFFER
resource_type => INT8
resource_name => COMPACT_STRING
configs => name config_operation value TAG_BUFFER
name => COMPACT_STRING
config_operation => INT8
value => COMPACT_NULLABLE_STRING
validate_only => BOOLEAN
Field |
Description |
---|
resources | The incremental updates for each resource. |
resource_type | The resource type. |
resource_name | The resource name. |
configs | The configurations. |
name | The configuration key name. |
config_operation | The type (Set, Delete, Append, Subtract) of operation. |
value | The value to set for the configuration key. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
validate_only | True if we should validate the request, but not change the configurations. |
_tagged_fields | The 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
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The responses for each resource. |
error_code | The resource error code. |
error_message | The resource error message, or null if there was no error. |
resource_type | The resource type. |
resource_name | The resource name. |
IncrementalAlterConfigs Response (Version: 1) => throttle_time_ms [responses] TAG_BUFFER
throttle_time_ms => INT32
responses => error_code error_message resource_type resource_name TAG_BUFFER
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
resource_type => INT8
resource_name => COMPACT_STRING
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The responses for each resource. |
error_code | The resource error code. |
error_message | The resource error message, or null if there was no error. |
resource_type | The resource type. |
resource_name | The resource name. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
AlterPartitionReassignments Request (Version: 0) => timeout_ms [topics] TAG_BUFFER
timeout_ms => INT32
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index [replicas] TAG_BUFFER
partition_index => INT32
replicas => INT32
Field |
Description |
---|
timeout_ms | The time in ms to wait for the request to complete. |
topics | The topics to reassign. |
name | The topic name. |
partitions | The partitions to reassign. |
partition_index | The partition index. |
replicas | The replicas to place the partitions on, or null to cancel a pending reassignment for this partition. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
AlterPartitionReassignments Response (Version: 0) => throttle_time_ms error_code error_message [responses] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
responses => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index error_code error_message TAG_BUFFER
partition_index => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | The 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_code | The top-level error code, or 0 if there was no error. |
error_message | The top-level error message, or null if there was no error. |
responses | The responses to topics to reassign. |
name | The topic name |
partitions | The responses to partitions to reassign |
partition_index | The partition index. |
error_code | The error code for this partition, or 0 if there was no error. |
error_message | The error message for this partition, or null if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
ListPartitionReassignments Request (Version: 0) => timeout_ms [topics] TAG_BUFFER
timeout_ms => INT32
topics => name [partition_indexes] TAG_BUFFER
name => COMPACT_STRING
partition_indexes => INT32
Field |
Description |
---|
timeout_ms | The time in ms to wait for the request to complete. |
topics | The topics to list partition reassignments for, or null to list everything. |
name | The topic name |
partition_indexes | The partitions to list partition reassignments for. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
ListPartitionReassignments Response (Version: 0) => throttle_time_ms error_code error_message [topics] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index [replicas] [adding_replicas] [removing_replicas] TAG_BUFFER
partition_index => INT32
replicas => INT32
adding_replicas => INT32
removing_replicas => INT32
Field |
Description |
---|
throttle_time_ms | The 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_code | The top-level error code, or 0 if there was no error |
error_message | The top-level error message, or null if there was no error. |
topics | The ongoing reassignments for each topic. |
name | The topic name. |
partitions | The ongoing reassignments for each partition. |
partition_index | The index of the partition. |
replicas | The current replica set. |
adding_replicas | The set of replicas we are currently adding. |
removing_replicas | The set of replicas we are currently removing. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
OffsetDelete Request (Version: 0) => group_id [topics]
group_id => STRING
topics => name [partitions]
name => STRING
partitions => partition_index
partition_index => INT32
Field |
Description |
---|
group_id | The unique group identifier. |
topics | The topics to delete offsets for |
name | The topic name. |
partitions | Each partition to delete offsets for. |
partition_index | The partition index. |
Responses:
OffsetDelete Response (Version: 0) => error_code throttle_time_ms [topics]
error_code => INT16
throttle_time_ms => INT32
topics => name [partitions]
name => STRING
partitions => partition_index error_code
partition_index => INT32
error_code => INT16
Field |
Description |
---|
error_code | The top-level error code, or 0 if there was no error. |
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses for each topic. |
name | The topic name. |
partitions | The responses for each partition in the topic. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no error. |
Requests:
DescribeClientQuotas Request (Version: 0) => [components] strict
components => entity_type match_type match
entity_type => STRING
match_type => INT8
match => NULLABLE_STRING
strict => BOOLEAN
Field |
Description |
---|
components | Filter components to apply to quota entities. |
entity_type | The entity type that the filter component applies to. |
match_type | How to match the entity {0 = exact name, 1 = default name, 2 = any specified name}. |
match | The string to match against, or null if unused for the match type. |
strict | Whether the match is strict, i.e. should exclude entities with unspecified entity types. |
DescribeClientQuotas Request (Version: 1) => [components] strict TAG_BUFFER
components => entity_type match_type match TAG_BUFFER
entity_type => COMPACT_STRING
match_type => INT8
match => COMPACT_NULLABLE_STRING
strict => BOOLEAN
Field |
Description |
---|
components | Filter components to apply to quota entities. |
entity_type | The entity type that the filter component applies to. |
match_type | How to match the entity {0 = exact name, 1 = default name, 2 = any specified name}. |
match | The string to match against, or null if unused for the match type. |
_tagged_fields | The tagged fields |
strict | Whether the match is strict, i.e. should exclude entities with unspecified entity types. |
_tagged_fields | The 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
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or `0` if the quota description succeeded. |
error_message | The error message, or `null` if the quota description succeeded. |
entries | A result entry. |
entity | The quota entity description. |
entity_type | The entity type. |
entity_name | The entity name, or null if the default. |
values | The quota values for the entity. |
key | The quota configuration key. |
value | The quota configuration value. |
DescribeClientQuotas Response (Version: 1) => throttle_time_ms error_code error_message [entries] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
entries => [entity] [values] TAG_BUFFER
entity => entity_type entity_name TAG_BUFFER
entity_type => COMPACT_STRING
entity_name => COMPACT_NULLABLE_STRING
values => key value TAG_BUFFER
key => COMPACT_STRING
value => FLOAT64
Field |
Description |
---|
throttle_time_ms | The 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_code | The error code, or `0` if the quota description succeeded. |
error_message | The error message, or `null` if the quota description succeeded. |
entries | A result entry. |
entity | The quota entity description. |
entity_type | The entity type. |
entity_name | The entity name, or null if the default. |
_tagged_fields | The tagged fields |
values | The quota values for the entity. |
key | The quota configuration key. |
value | The quota configuration value. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
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
Field |
Description |
---|
entries | The quota configuration entries to alter. |
entity | The quota entity to alter. |
entity_type | The entity type. |
entity_name | The name of the entity, or null if the default. |
ops | An individual quota configuration entry to alter. |
key | The quota configuration key. |
value | The value to set, otherwise ignored if the value is to be removed. |
remove | Whether the quota configuration value should be removed, otherwise set. |
validate_only | Whether the alteration should be validated, but not performed. |
AlterClientQuotas Request (Version: 1) => [entries] validate_only TAG_BUFFER
entries => [entity] [ops] TAG_BUFFER
entity => entity_type entity_name TAG_BUFFER
entity_type => COMPACT_STRING
entity_name => COMPACT_NULLABLE_STRING
ops => key value remove TAG_BUFFER
key => COMPACT_STRING
value => FLOAT64
remove => BOOLEAN
validate_only => BOOLEAN
Field |
Description |
---|
entries | The quota configuration entries to alter. |
entity | The quota entity to alter. |
entity_type | The entity type. |
entity_name | The name of the entity, or null if the default. |
_tagged_fields | The tagged fields |
ops | An individual quota configuration entry to alter. |
key | The quota configuration key. |
value | The value to set, otherwise ignored if the value is to be removed. |
remove | Whether the quota configuration value should be removed, otherwise set. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
validate_only | Whether the alteration should be validated, but not performed. |
_tagged_fields | The 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
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
entries | The quota configuration entries to alter. |
error_code | The error code, or `0` if the quota alteration succeeded. |
error_message | The error message, or `null` if the quota alteration succeeded. |
entity | The quota entity to alter. |
entity_type | The entity type. |
entity_name | The name of the entity, or null if the default. |
AlterClientQuotas Response (Version: 1) => throttle_time_ms [entries] TAG_BUFFER
throttle_time_ms => INT32
entries => error_code error_message [entity] TAG_BUFFER
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
entity => entity_type entity_name TAG_BUFFER
entity_type => COMPACT_STRING
entity_name => COMPACT_NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
entries | The quota configuration entries to alter. |
error_code | The error code, or `0` if the quota alteration succeeded. |
error_message | The error message, or `null` if the quota alteration succeeded. |
entity | The quota entity to alter. |
entity_type | The entity type. |
entity_name | The name of the entity, or null if the default. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
DescribeUserScramCredentials Request (Version: 0) => [users] TAG_BUFFER
users => name TAG_BUFFER
name => COMPACT_STRING
Field |
Description |
---|
users | The users to describe, or null/empty to describe all users. |
name | The user name. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
DescribeUserScramCredentials Response (Version: 0) => throttle_time_ms error_code error_message [results] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
results => user error_code error_message [credential_infos] TAG_BUFFER
user => COMPACT_STRING
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
credential_infos => mechanism iterations TAG_BUFFER
mechanism => INT8
iterations => INT32
Field |
Description |
---|
throttle_time_ms | The 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_code | The message-level error code, 0 except for user authorization or infrastructure issues. |
error_message | The message-level error message, if any. |
results | The results for descriptions, one per user. |
user | The user name. |
error_code | The user-level error code. |
error_message | The user-level error message, if any. |
credential_infos | The mechanism and related information associated with the user's SCRAM credentials. |
mechanism | The SCRAM mechanism. |
iterations | The number of iterations used in the SCRAM credential. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
AlterUserScramCredentials Request (Version: 0) => [deletions] [upsertions] TAG_BUFFER
deletions => name mechanism TAG_BUFFER
name => COMPACT_STRING
mechanism => INT8
upsertions => name mechanism iterations salt salted_password TAG_BUFFER
name => COMPACT_STRING
mechanism => INT8
iterations => INT32
salt => COMPACT_BYTES
salted_password => COMPACT_BYTES
Field |
Description |
---|
deletions | The SCRAM credentials to remove. |
name | The user name. |
mechanism | The SCRAM mechanism. |
_tagged_fields | The tagged fields |
upsertions | The SCRAM credentials to update/insert. |
name | The user name. |
mechanism | The SCRAM mechanism. |
iterations | The number of iterations. |
salt | A random salt generated by the client. |
salted_password | The salted password. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
AlterUserScramCredentials Response (Version: 0) => throttle_time_ms [results] TAG_BUFFER
throttle_time_ms => INT32
results => user error_code error_message TAG_BUFFER
user => COMPACT_STRING
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for deletions and alterations, one per affected user. |
user | The user name. |
error_code | The error code. |
error_message | The error message, if any. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
DescribeQuorum Request (Version: 0) => [topics] TAG_BUFFER
topics => topic_name [partitions] TAG_BUFFER
topic_name => COMPACT_STRING
partitions => partition_index TAG_BUFFER
partition_index => INT32
Field |
Description |
---|
topics | |
topic_name | The topic name. |
partitions | |
partition_index | The partition index. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DescribeQuorum Request (Version: 1) => [topics] TAG_BUFFER
topics => topic_name [partitions] TAG_BUFFER
topic_name => COMPACT_STRING
partitions => partition_index TAG_BUFFER
partition_index => INT32
Field |
Description |
---|
topics | |
topic_name | The topic name. |
partitions | |
partition_index | The partition index. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
DescribeQuorum Response (Version: 0) => error_code [topics] TAG_BUFFER
error_code => INT16
topics => topic_name [partitions] TAG_BUFFER
topic_name => COMPACT_STRING
partitions => partition_index error_code leader_id leader_epoch high_watermark [current_voters] [observers] TAG_BUFFER
partition_index => INT32
error_code => INT16
leader_id => INT32
leader_epoch => INT32
high_watermark => INT64
current_voters => replica_id log_end_offset TAG_BUFFER
replica_id => INT32
log_end_offset => INT64
observers => replica_id log_end_offset TAG_BUFFER
replica_id => INT32
log_end_offset => INT64
Field |
Description |
---|
error_code | The top level error code. |
topics | |
topic_name | The topic name. |
partitions | |
partition_index | The partition index. |
error_code | |
leader_id | The ID of the current leader or -1 if the leader is unknown. |
leader_epoch | The latest known leader epoch |
high_watermark | |
current_voters | |
replica_id | |
log_end_offset | The last known log end offset of the follower or -1 if it is unknown |
_tagged_fields | The tagged fields |
observers | |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DescribeQuorum Response (Version: 1) => error_code [topics] TAG_BUFFER
error_code => INT16
topics => topic_name [partitions] TAG_BUFFER
topic_name => COMPACT_STRING
partitions => partition_index error_code leader_id leader_epoch high_watermark [current_voters] [observers] TAG_BUFFER
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 TAG_BUFFER
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 TAG_BUFFER
replica_id => INT32
log_end_offset => INT64
last_fetch_timestamp => INT64
last_caught_up_timestamp => INT64
Field |
Description |
---|
error_code | The top level error code. |
topics | |
topic_name | The topic name. |
partitions | |
partition_index | The partition index. |
error_code | |
leader_id | The ID of the current leader or -1 if the leader is unknown. |
leader_epoch | The latest known leader epoch |
high_watermark | |
current_voters | |
replica_id | |
log_end_offset | The last known log end offset of the follower or -1 if it is unknown |
last_fetch_timestamp | The 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_timestamp | The 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_fields | The tagged fields |
observers | |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
AlterPartition Request (Version: 0) => broker_id broker_epoch [topics] TAG_BUFFER
broker_id => INT32
broker_epoch => INT64
topics => topic_name [partitions] TAG_BUFFER
topic_name => COMPACT_STRING
partitions => partition_index leader_epoch [new_isr] partition_epoch TAG_BUFFER
partition_index => INT32
leader_epoch => INT32
new_isr => INT32
partition_epoch => INT32
Field |
Description |
---|
broker_id | The ID of the requesting broker |
broker_epoch | The epoch of the requesting broker |
topics | |
topic_name | The name of the topic to alter ISRs for |
partitions | |
partition_index | The partition index |
leader_epoch | The leader epoch of this partition |
new_isr | The ISR for this partition |
partition_epoch | The expected epoch of the partition which is being updated. For legacy cluster this is the ZkVersion in the LeaderAndIsr request. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
AlterPartition Request (Version: 1) => broker_id broker_epoch [topics] TAG_BUFFER
broker_id => INT32
broker_epoch => INT64
topics => topic_name [partitions] TAG_BUFFER
topic_name => COMPACT_STRING
partitions => partition_index leader_epoch [new_isr] leader_recovery_state partition_epoch TAG_BUFFER
partition_index => INT32
leader_epoch => INT32
new_isr => INT32
leader_recovery_state => INT8
partition_epoch => INT32
Field |
Description |
---|
broker_id | The ID of the requesting broker |
broker_epoch | The epoch of the requesting broker |
topics | |
topic_name | The name of the topic to alter ISRs for |
partitions | |
partition_index | The partition index |
leader_epoch | The leader epoch of this partition |
new_isr | The ISR for this partition |
leader_recovery_state | 1 if the partition is recovering from an unclean leader election; 0 otherwise. |
partition_epoch | The expected epoch of the partition which is being updated. For legacy cluster this is the ZkVersion in the LeaderAndIsr request. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
AlterPartition Request (Version: 2) => broker_id broker_epoch [topics] TAG_BUFFER
broker_id => INT32
broker_epoch => INT64
topics => topic_id [partitions] TAG_BUFFER
topic_id => UUID
partitions => partition_index leader_epoch [new_isr] leader_recovery_state partition_epoch TAG_BUFFER
partition_index => INT32
leader_epoch => INT32
new_isr => INT32
leader_recovery_state => INT8
partition_epoch => INT32
Field |
Description |
---|
broker_id | The ID of the requesting broker |
broker_epoch | The epoch of the requesting broker |
topics | |
topic_id | The ID of the topic to alter ISRs for |
partitions | |
partition_index | The partition index |
leader_epoch | The leader epoch of this partition |
new_isr | The ISR for this partition |
leader_recovery_state | 1 if the partition is recovering from an unclean leader election; 0 otherwise. |
partition_epoch | The expected epoch of the partition which is being updated. For legacy cluster this is the ZkVersion in the LeaderAndIsr request. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
AlterPartition Response (Version: 0) => throttle_time_ms error_code [topics] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
topics => topic_name [partitions] TAG_BUFFER
topic_name => COMPACT_STRING
partitions => partition_index error_code leader_id leader_epoch [isr] partition_epoch TAG_BUFFER
partition_index => INT32
error_code => INT16
leader_id => INT32
leader_epoch => INT32
isr => INT32
partition_epoch => INT32
Field |
Description |
---|
throttle_time_ms | The 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_code | The top level response error code |
topics | |
topic_name | The name of the topic |
partitions | |
partition_index | The partition index |
error_code | The partition level error code |
leader_id | The broker ID of the leader. |
leader_epoch | The leader epoch. |
isr | The in-sync replica IDs. |
partition_epoch | The current epoch for the partition for KRaft controllers. The current ZK version for the legacy controllers. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
AlterPartition Response (Version: 1) => throttle_time_ms error_code [topics] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
topics => topic_name [partitions] TAG_BUFFER
topic_name => COMPACT_STRING
partitions => partition_index error_code leader_id leader_epoch [isr] leader_recovery_state partition_epoch TAG_BUFFER
partition_index => INT32
error_code => INT16
leader_id => INT32
leader_epoch => INT32
isr => INT32
leader_recovery_state => INT8
partition_epoch => INT32
Field |
Description |
---|
throttle_time_ms | The 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_code | The top level response error code |
topics | |
topic_name | The name of the topic |
partitions | |
partition_index | The partition index |
error_code | The partition level error code |
leader_id | The broker ID of the leader. |
leader_epoch | The leader epoch. |
isr | The in-sync replica IDs. |
leader_recovery_state | 1 if the partition is recovering from an unclean leader election; 0 otherwise. |
partition_epoch | The current epoch for the partition for KRaft controllers. The current ZK version for the legacy controllers. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
AlterPartition Response (Version: 2) => throttle_time_ms error_code [topics] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
topics => topic_id [partitions] TAG_BUFFER
topic_id => UUID
partitions => partition_index error_code leader_id leader_epoch [isr] leader_recovery_state partition_epoch TAG_BUFFER
partition_index => INT32
error_code => INT16
leader_id => INT32
leader_epoch => INT32
isr => INT32
leader_recovery_state => INT8
partition_epoch => INT32
Field |
Description |
---|
throttle_time_ms | The 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_code | The top level response error code |
topics | |
topic_id | The ID of the topic |
partitions | |
partition_index | The partition index |
error_code | The partition level error code |
leader_id | The broker ID of the leader. |
leader_epoch | The leader epoch. |
isr | The in-sync replica IDs. |
leader_recovery_state | 1 if the partition is recovering from an unclean leader election; 0 otherwise. |
partition_epoch | The current epoch for the partition for KRaft controllers. The current ZK version for the legacy controllers. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
UpdateFeatures Request (Version: 0) => timeout_ms [feature_updates] TAG_BUFFER
timeout_ms => INT32
feature_updates => feature max_version_level allow_downgrade TAG_BUFFER
feature => COMPACT_STRING
max_version_level => INT16
allow_downgrade => BOOLEAN
Field |
Description |
---|
timeout_ms | How long to wait in milliseconds before timing out the request. |
feature_updates | The list of updates to finalized features. |
feature | The name of the finalized feature to be updated. |
max_version_level | The 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_downgrade | DEPRECATED 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_fields | The tagged fields |
_tagged_fields | The tagged fields |
UpdateFeatures Request (Version: 1) => timeout_ms [feature_updates] validate_only TAG_BUFFER
timeout_ms => INT32
feature_updates => feature max_version_level upgrade_type TAG_BUFFER
feature => COMPACT_STRING
max_version_level => INT16
upgrade_type => INT8
validate_only => BOOLEAN
Field |
Description |
---|
timeout_ms | How long to wait in milliseconds before timing out the request. |
feature_updates | The list of updates to finalized features. |
feature | The name of the finalized feature to be updated. |
max_version_level | The 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_type | Determine 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_fields | The tagged fields |
validate_only | True if we should validate the request, but not perform the upgrade or downgrade. |
_tagged_fields | The tagged fields |
Responses:
UpdateFeatures Response (Version: 0) => throttle_time_ms error_code error_message [results] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
results => feature error_code error_message TAG_BUFFER
feature => COMPACT_STRING
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | The 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_code | The top-level error code, or `0` if there was no top-level error. |
error_message | The top-level error message, or `null` if there was no top-level error. |
results | Results for each feature update. |
feature | The name of the finalized feature. |
error_code | The feature update error code or `0` if the feature update succeeded. |
error_message | The feature update error, or `null` if the feature update succeeded. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
UpdateFeatures Response (Version: 1) => throttle_time_ms error_code error_message [results] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
results => feature error_code error_message TAG_BUFFER
feature => COMPACT_STRING
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | The 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_code | The top-level error code, or `0` if there was no top-level error. |
error_message | The top-level error message, or `null` if there was no top-level error. |
results | Results for each feature update. |
feature | The name of the finalized feature. |
error_code | The feature update error code or `0` if the feature update succeeded. |
error_message | The feature update error, or `null` if the feature update succeeded. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
Envelope Request (Version: 0) => request_data request_principal client_host_address TAG_BUFFER
request_data => COMPACT_BYTES
request_principal => COMPACT_NULLABLE_BYTES
client_host_address => COMPACT_BYTES
Field |
Description |
---|
request_data | The embedded request header and data. |
request_principal | Value of the initial client principal when the request is redirected by a broker. |
client_host_address | The original client's address in bytes. |
_tagged_fields | The tagged fields |
Responses:
Envelope Response (Version: 0) => response_data error_code TAG_BUFFER
response_data => COMPACT_NULLABLE_BYTES
error_code => INT16
Field |
Description |
---|
response_data | The embedded response header and data. |
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
Requests:
DescribeCluster Request (Version: 0) => include_cluster_authorized_operations TAG_BUFFER
include_cluster_authorized_operations => BOOLEAN
Field |
Description |
---|
include_cluster_authorized_operations | Whether to include cluster authorized operations. |
_tagged_fields | The tagged fields |
Responses:
DescribeCluster Response (Version: 0) => throttle_time_ms error_code error_message cluster_id controller_id [brokers] cluster_authorized_operations TAG_BUFFER
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 TAG_BUFFER
broker_id => INT32
host => COMPACT_STRING
port => INT32
rack => COMPACT_NULLABLE_STRING
cluster_authorized_operations => INT32
Field |
Description |
---|
throttle_time_ms | The 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_code | The top-level error code, or 0 if there was no error |
error_message | The top-level error message, or null if there was no error. |
cluster_id | The cluster ID that responding broker belongs to. |
controller_id | The ID of the controller broker. |
brokers | Each broker in the response. |
broker_id | The broker ID. |
host | The broker hostname. |
port | The broker port. |
rack | The rack of the broker, or null if it has not been assigned to a rack. |
_tagged_fields | The tagged fields |
cluster_authorized_operations | 32-bit bitfield to represent authorized operations for this cluster. |
_tagged_fields | The tagged fields |
Requests:
DescribeProducers Request (Version: 0) => [topics] TAG_BUFFER
topics => name [partition_indexes] TAG_BUFFER
name => COMPACT_STRING
partition_indexes => INT32
Field |
Description |
---|
topics | |
name | The topic name. |
partition_indexes | The indexes of the partitions to list producers for. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Responses:
DescribeProducers Response (Version: 0) => throttle_time_ms [topics] TAG_BUFFER
throttle_time_ms => INT32
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index error_code error_message [active_producers] TAG_BUFFER
partition_index => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
active_producers => producer_id producer_epoch last_sequence last_timestamp coordinator_epoch current_txn_start_offset TAG_BUFFER
producer_id => INT64
producer_epoch => INT32
last_sequence => INT32
last_timestamp => INT64
coordinator_epoch => INT32
current_txn_start_offset => INT64
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Each topic in the response. |
name | The topic name |
partitions | Each partition in the response. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
error_message | The partition error message, which may be null if no additional details are available |
active_producers | |
producer_id | |
producer_epoch | |
last_sequence | |
last_timestamp | |
coordinator_epoch | |
current_txn_start_offset | |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
UnregisterBroker Request (Version: 0) => broker_id TAG_BUFFER
broker_id => INT32
Field |
Description |
---|
broker_id | The broker ID to unregister. |
_tagged_fields | The tagged fields |
Responses:
UnregisterBroker Response (Version: 0) => throttle_time_ms error_code error_message TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | 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_code | The error code, or 0 if there was no error. |
error_message | The top-level error message, or `null` if there was no top-level error. |
_tagged_fields | The tagged fields |
Requests:
DescribeTransactions Request (Version: 0) => [transactional_ids] TAG_BUFFER
transactional_ids => COMPACT_STRING
Field |
Description |
---|
transactional_ids | Array of transactionalIds to include in describe results. If empty, then no results will be returned. |
_tagged_fields | The tagged fields |
Responses:
DescribeTransactions Response (Version: 0) => throttle_time_ms [transaction_states] TAG_BUFFER
throttle_time_ms => INT32
transaction_states => error_code transactional_id transaction_state transaction_timeout_ms transaction_start_time_ms producer_id producer_epoch [topics] TAG_BUFFER
error_code => INT16
transactional_id => COMPACT_STRING
transaction_state => COMPACT_STRING
transaction_timeout_ms => INT32
transaction_start_time_ms => INT64
producer_id => INT64
producer_epoch => INT16
topics => topic [partitions] TAG_BUFFER
topic => COMPACT_STRING
partitions => INT32
Field |
Description |
---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
transaction_states | |
error_code | |
transactional_id | |
transaction_state | |
transaction_timeout_ms | |
transaction_start_time_ms | |
producer_id | |
producer_epoch | |
topics | The set of partitions included in the current transaction (if active). When a transaction is preparing to commit or abort, this will include only partitions which do not have markers. |
topic | |
partitions | |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
ListTransactions Request (Version: 0) => [state_filters] [producer_id_filters] TAG_BUFFER
state_filters => COMPACT_STRING
producer_id_filters => INT64
Field |
Description |
---|
state_filters | The 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_filters | The 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_fields | The tagged fields |
Responses:
ListTransactions Response (Version: 0) => throttle_time_ms error_code [unknown_state_filters] [transaction_states] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
unknown_state_filters => COMPACT_STRING
transaction_states => transactional_id producer_id transaction_state TAG_BUFFER
transactional_id => COMPACT_STRING
producer_id => INT64
transaction_state => COMPACT_STRING
Field |
Description |
---|
throttle_time_ms | The 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_code | |
unknown_state_filters | Set of state filters provided in the request which were unknown to the transaction coordinator |
transaction_states | |
transactional_id | |
producer_id | |
transaction_state | The current transaction state of the producer |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Requests:
AllocateProducerIds Request (Version: 0) => broker_id broker_epoch TAG_BUFFER
broker_id => INT32
broker_epoch => INT64
Field |
Description |
---|
broker_id | The ID of the requesting broker |
broker_epoch | The epoch of the requesting broker |
_tagged_fields | The tagged fields |
Responses:
AllocateProducerIds Response (Version: 0) => throttle_time_ms error_code producer_id_start producer_id_len TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
producer_id_start => INT64
producer_id_len => INT32
Field |
Description |
---|
throttle_time_ms | The 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_code | The top level response error code |
producer_id_start | The first producer ID in this range, inclusive |
producer_id_len | The number of producer IDs in this range |
_tagged_fields | The tagged fields |
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.