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.
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.
The protocol is designed to enable incremental evolution in a backward compatible fashion. Our versioning is on a per API basis, each version consisting of a request and response pair. Each request contains an API key that identifies the API being invoked and a version number that indicates the format of the request and the expected format of the response.
The intention is that clients will support a range of API versions. When communicating with a particular broker, a given client should use the highest API version supported by both and indicate this version in their requests.
The server will reject requests with a version it does not support, and will always respond to the client with exactly the protocol format it expects based on the version it included in its request. The intended upgrade path is that new features would first be rolled out on the server (with the older clients not making use of them) and then as newer clients are deployed these new features would gradually be taken advantage of.
Our goal is primarily to allow API evolution in an environment where downtime is not allowed and clients and servers cannot all be changed at once.
Currently all versions are baselined at 0, as we evolve these APIs we will indicate the format for each version individually.
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 a SSL listener before the SSL handshake is finished). If this is considered to
leak information about the broker version a workaround is to use SSL with client authentication which is performed at an
earlier stage of the connection where the ApiVersionRequest
is not available. Also, note that broker versions older
than 0.10.0.0 do not support this API and will either ignore the request or close connection in response to the request.
- 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.
Fixed Width Primitives
int8, int16, int32, int64 - Signed integers with the given precision (in bits) stored in big endian order.
Variable Length Primitives
bytes, string - These types consist of a signed integer giving a length N followed by N bytes of content. A length of -1 indicates null. string uses an int16 for its size, and bytes uses an int32.
Arrays
This is a notation for handling repeated structures. These will always be encoded as an int32 size containing the length N followed by N repetitions of the structure which can itself be made up of other primitive types. In the BNF grammars below we will show an array of a structure foo as [foo].
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 message set format can be found here. (KAFKA-3368)
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, or is otherwise corrupt. |
UNKNOWN_TOPIC_OR_PARTITION | 3 | True | This server does not host this topic-partition. |
INVALID_FETCH_SIZE | 4 | False | The requested fetch size is invalid. |
LEADER_NOT_AVAILABLE | 5 | True | There is no leader for this topic-partition as we are in the middle of a leadership election. |
NOT_LEADER_FOR_PARTITION | 6 | True | This server is not the leader for that topic-partition. |
REQUEST_TIMED_OUT | 7 | True | The request timed out. |
BROKER_NOT_AVAILABLE | 8 | False | The broker is not available. |
REPLICA_NOT_AVAILABLE | 9 | False | The replica is not available for the requested topic-partition |
MESSAGE_TOO_LARGE | 10 | False | The request included a message larger than the max message size the server will accept. |
STALE_CONTROLLER_EPOCH | 11 | False | The controller moved to another broker. |
OFFSET_METADATA_TOO_LARGE | 12 | False | The metadata field of the offset request was too large. |
NETWORK_EXCEPTION | 13 | True | The server disconnected before a response was received. |
COORDINATOR_LOAD_IN_PROGRESS | 14 | True | The coordinator is loading and hence can't process requests. |
COORDINATOR_NOT_AVAILABLE | 15 | True | The coordinator is not available. |
NOT_COORDINATOR | 16 | True | This is not the correct coordinator. |
INVALID_TOPIC_EXCEPTION | 17 | False | The request attempted to perform an operation on an invalid topic. |
RECORD_LIST_TOO_LARGE | 18 | False | The request included message batch larger than the configured segment size on the server. |
NOT_ENOUGH_REPLICAS | 19 | True | Messages are rejected since there are fewer in-sync replicas than required. |
NOT_ENOUGH_REPLICAS_AFTER_APPEND | 20 | True | Messages are written to the log, but to fewer in-sync replicas than required. |
INVALID_REQUIRED_ACKS | 21 | False | Produce request specified an invalid value for required acks. |
ILLEGAL_GENERATION | 22 | False | Specified group generation id is not valid. |
INCONSISTENT_GROUP_PROTOCOL | 23 | False | The group member's supported protocols are incompatible with those of existing members 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 | Not authorized to access topics: [Topic authorization failed.] |
GROUP_AUTHORIZATION_FAILED | 30 | False | Not authorized to access group: Group authorization failed. |
CLUSTER_AUTHORIZATION_FAILED | 31 | False | Cluster authorization failed. |
INVALID_TIMESTAMP | 32 | False | The timestamp of the message is out of acceptable range. |
UNSUPPORTED_SASL_MECHANISM | 33 | False | The broker does not support the requested SASL mechanism. |
ILLEGAL_SASL_STATE | 34 | False | Request is not valid given the current SASL state. |
UNSUPPORTED_VERSION | 35 | False | The version of API is not supported. |
TOPIC_ALREADY_EXISTS | 36 | False | Topic with this name already exists. |
INVALID_PARTITIONS | 37 | False | Number of partitions is invalid. |
INVALID_REPLICATION_FACTOR | 38 | False | Replication-factor is invalid. |
INVALID_REPLICA_ASSIGNMENT | 39 | False | Replica assignment is invalid. |
INVALID_CONFIG | 40 | False | Configuration is invalid. |
NOT_CONTROLLER | 41 | True | This is not the correct controller for this cluster. |
INVALID_REQUEST | 42 | False | This most likely occurs because of a request being malformed by the client library or the message was sent to an incompatible broker. See the broker logs for more details. |
UNSUPPORTED_FOR_MESSAGE_FORMAT | 43 | False | The message format version on the broker does not support the request. |
POLICY_VIOLATION | 44 | False | Request parameters do not satisfy the configured policy. |
OUT_OF_ORDER_SEQUENCE_NUMBER | 45 | False | The broker received an out of order sequence number |
DUPLICATE_SEQUENCE_NUMBER | 46 | False | The broker received a duplicate sequence number |
INVALID_PRODUCER_EPOCH | 47 | False | Producer attempted an operation with an old epoch. Either there is a newer producer with the same transactionalId, or the producer's transaction has been expired by the broker. |
INVALID_TXN_STATE | 48 | False | The producer attempted a transactional operation in an invalid state |
INVALID_PRODUCER_ID_MAPPING | 49 | False | The producer attempted to use a producer id which is not currently assigned to its transactional id |
INVALID_TRANSACTION_TIMEOUT | 50 | False | The transaction timeout is larger than the maximum value allowed by the broker (as configured by transaction.max.timeout.ms). |
CONCURRENT_TRANSACTIONS | 51 | False | The producer attempted to update a transaction while another concurrent operation on the same transaction was ongoing |
TRANSACTION_COORDINATOR_FENCED | 52 | False | Indicates that the transaction coordinator sending a WriteTxnMarker is no longer the current coordinator for a given producer |
TRANSACTIONAL_ID_AUTHORIZATION_FAILED | 53 | False | Transactional Id authorization failed |
SECURITY_DISABLED | 54 | False | Security features are disabled. |
OPERATION_NOT_ATTEMPTED | 55 | False | The broker did not attempt to execute this operation. This may happen for batched RPCs where some operations in the batch failed, causing the broker to respond without trying the rest. |
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 The group is not empty is not empty |
GROUP_ID_NOT_FOUND | 69 | False | The group id The group id does not exist was not found |
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 |
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 => api_key api_version correlation_id client_id
api_key => INT16
api_version => INT16
correlation_id => INT32
client_id => NULLABLE_STRING
Field |
Description |
---|
api_key | The id of the request type. |
api_version | The version of the API. |
correlation_id | A user-supplied integer value that will be passed back with the response |
client_id | A user specified identifier for the client making the request. |
Response Header => correlation_id
correlation_id => INT32
Field |
Description |
---|
correlation_id | The user-supplied value passed in with the request |
Requests:
Produce Request (Version: 0) => acks timeout [topic_data]
acks => INT16
timeout => INT32
topic_data => topic [data]
topic => STRING
data => partition record_set
partition => INT32
record_set => RECORDS
Field |
Description |
---|
acks | The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR. |
timeout | The time to await a response in ms. |
topic_data | null |
topic | Name of topic |
data | null |
partition | Topic partition id |
record_set | null |
Produce Request (Version: 1) => acks timeout [topic_data]
acks => INT16
timeout => INT32
topic_data => topic [data]
topic => STRING
data => partition record_set
partition => INT32
record_set => RECORDS
Field |
Description |
---|
acks | The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR. |
timeout | The time to await a response in ms. |
topic_data | null |
topic | Name of topic |
data | null |
partition | Topic partition id |
record_set | null |
Produce Request (Version: 2) => acks timeout [topic_data]
acks => INT16
timeout => INT32
topic_data => topic [data]
topic => STRING
data => partition record_set
partition => INT32
record_set => RECORDS
Field |
Description |
---|
acks | The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR. |
timeout | The time to await a response in ms. |
topic_data | null |
topic | Name of topic |
data | null |
partition | Topic partition id |
record_set | null |
Produce Request (Version: 3) => transactional_id acks timeout [topic_data]
transactional_id => NULLABLE_STRING
acks => INT16
timeout => INT32
topic_data => topic [data]
topic => STRING
data => partition record_set
partition => INT32
record_set => RECORDS
Field |
Description |
---|
transactional_id | The transactional id 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 | The time to await a response in ms. |
topic_data | null |
topic | Name of topic |
data | null |
partition | Topic partition id |
record_set | null |
Produce Request (Version: 4) => transactional_id acks timeout [topic_data]
transactional_id => NULLABLE_STRING
acks => INT16
timeout => INT32
topic_data => topic [data]
topic => STRING
data => partition record_set
partition => INT32
record_set => RECORDS
Field |
Description |
---|
transactional_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 | The time to await a response in ms. |
topic_data | null |
topic | Name of topic |
data | null |
partition | Topic partition id |
record_set | null |
Produce Request (Version: 5) => transactional_id acks timeout [topic_data]
transactional_id => NULLABLE_STRING
acks => INT16
timeout => INT32
topic_data => topic [data]
topic => STRING
data => partition record_set
partition => INT32
record_set => RECORDS
Field |
Description |
---|
transactional_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 | The time to await a response in ms. |
topic_data | null |
topic | Name of topic |
data | null |
partition | Topic partition id |
record_set | null |
Responses:
Produce Response (Version: 0) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code base_offset
partition => INT32
error_code => INT16
base_offset => INT64
Field |
Description |
---|
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
error_code | Response error code |
base_offset | null |
Produce Response (Version: 1) => [responses] throttle_time_ms
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code base_offset
partition => INT32
error_code => INT16
base_offset => INT64
throttle_time_ms => INT32
Field |
Description |
---|
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
error_code | Response error code |
base_offset | null |
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
Produce Response (Version: 2) => [responses] throttle_time_ms
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code base_offset log_append_time
partition => INT32
error_code => INT16
base_offset => INT64
log_append_time => INT64
throttle_time_ms => INT32
Field |
Description |
---|
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
error_code | Response error code |
base_offset | null |
log_append_time | The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended. |
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
Produce Response (Version: 3) => [responses] throttle_time_ms
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code base_offset log_append_time
partition => INT32
error_code => INT16
base_offset => INT64
log_append_time => INT64
throttle_time_ms => INT32
Field |
Description |
---|
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
error_code | Response error code |
base_offset | null |
log_append_time | The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended. |
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
Produce Response (Version: 4) => [responses] throttle_time_ms
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code base_offset log_append_time
partition => INT32
error_code => INT16
base_offset => INT64
log_append_time => INT64
throttle_time_ms => INT32
Field |
Description |
---|
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
error_code | Response error code |
base_offset | null |
log_append_time | The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended. |
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
Produce Response (Version: 5) => [responses] throttle_time_ms
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code base_offset log_append_time log_start_offset
partition => INT32
error_code => INT16
base_offset => INT64
log_append_time => INT64
log_start_offset => INT64
throttle_time_ms => INT32
Field |
Description |
---|
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
error_code | Response error code |
base_offset | null |
log_append_time | The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended. |
log_start_offset | The start offset of the log at the time this produce response was created |
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
Requests:
Fetch Request (Version: 0) => replica_id max_wait_time min_bytes [topics]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset max_bytes
partition => INT32
fetch_offset => INT64
max_bytes => INT32
Field |
Description |
---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
max_wait_time | Maximum time in ms to wait for the response. |
min_bytes | Minimum bytes to accumulate in the response. |
topics | Topics to fetch. |
topic | Name of topic |
partitions | Partitions to fetch. |
partition | Topic partition id |
fetch_offset | Message offset. |
max_bytes | Maximum bytes to fetch. |
Fetch Request (Version: 1) => replica_id max_wait_time min_bytes [topics]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset max_bytes
partition => INT32
fetch_offset => INT64
max_bytes => INT32
Field |
Description |
---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
max_wait_time | Maximum time in ms to wait for the response. |
min_bytes | Minimum bytes to accumulate in the response. |
topics | Topics to fetch. |
topic | Name of topic |
partitions | Partitions to fetch. |
partition | Topic partition id |
fetch_offset | Message offset. |
max_bytes | Maximum bytes to fetch. |
Fetch Request (Version: 2) => replica_id max_wait_time min_bytes [topics]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset max_bytes
partition => INT32
fetch_offset => INT64
max_bytes => INT32
Field |
Description |
---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
max_wait_time | Maximum time in ms to wait for the response. |
min_bytes | Minimum bytes to accumulate in the response. |
topics | Topics to fetch. |
topic | Name of topic |
partitions | Partitions to fetch. |
partition | Topic partition id |
fetch_offset | Message offset. |
max_bytes | Maximum bytes to fetch. |
Fetch Request (Version: 3) => replica_id max_wait_time min_bytes max_bytes [topics]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
max_bytes => INT32
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset max_bytes
partition => INT32
fetch_offset => INT64
max_bytes => INT32
Field |
Description |
---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
max_wait_time | Maximum time in ms to wait for the response. |
min_bytes | Minimum bytes to accumulate in the response. |
max_bytes | Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made. |
topics | Topics to fetch in the order provided. |
topic | Name of topic |
partitions | Partitions to fetch. |
partition | Topic partition id |
fetch_offset | Message offset. |
max_bytes | Maximum bytes to fetch. |
Fetch Request (Version: 4) => replica_id max_wait_time min_bytes max_bytes isolation_level [topics]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
max_bytes => INT32
isolation_level => INT8
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset max_bytes
partition => INT32
fetch_offset => INT64
max_bytes => INT32
Field |
Description |
---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
max_wait_time | Maximum time in ms to wait for the response. |
min_bytes | Minimum bytes to accumulate in the response. |
max_bytes | Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
topics | Topics to fetch in the order provided. |
topic | Name of topic |
partitions | Partitions to fetch. |
partition | Topic partition id |
fetch_offset | Message offset. |
max_bytes | Maximum bytes to fetch. |
Fetch Request (Version: 5) => replica_id max_wait_time min_bytes max_bytes isolation_level [topics]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
max_bytes => INT32
isolation_level => INT8
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset log_start_offset max_bytes
partition => INT32
fetch_offset => INT64
log_start_offset => INT64
max_bytes => INT32
Field |
Description |
---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
max_wait_time | Maximum time in ms to wait for the response. |
min_bytes | Minimum bytes to accumulate in the response. |
max_bytes | Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
topics | Topics to fetch in the order provided. |
topic | Name of topic |
partitions | Partitions to fetch. |
partition | Topic partition id |
fetch_offset | Message offset. |
log_start_offset | Earliest available offset of the follower replica. The field is only used when request is sent by follower. |
max_bytes | Maximum bytes to fetch. |
Fetch Request (Version: 6) => replica_id max_wait_time min_bytes max_bytes isolation_level [topics]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
max_bytes => INT32
isolation_level => INT8
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset log_start_offset max_bytes
partition => INT32
fetch_offset => INT64
log_start_offset => INT64
max_bytes => INT32
Field |
Description |
---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
max_wait_time | Maximum time in ms to wait for the response. |
min_bytes | Minimum bytes to accumulate in the response. |
max_bytes | Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
topics | Topics to fetch in the order provided. |
topic | Name of topic |
partitions | Partitions to fetch. |
partition | Topic partition id |
fetch_offset | Message offset. |
log_start_offset | Earliest available offset of the follower replica. The field is only used when request is sent by follower. |
max_bytes | Maximum bytes to fetch. |
Fetch Request (Version: 7) => replica_id max_wait_time min_bytes max_bytes isolation_level session_id epoch [topics] [forgetten_topics_data]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
max_bytes => INT32
isolation_level => INT8
session_id => INT32
epoch => INT32
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset log_start_offset max_bytes
partition => INT32
fetch_offset => INT64
log_start_offset => INT64
max_bytes => INT32
forgetten_topics_data => topic [partitions]
topic => STRING
partitions => INT32
Field |
Description |
---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
max_wait_time | Maximum time in ms to wait for the response. |
min_bytes | Minimum bytes to accumulate in the response. |
max_bytes | Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
session_id | The fetch session ID |
epoch | The fetch epoch |
topics | Topics to fetch in the order provided. |
topic | Name of topic |
partitions | Partitions to fetch. |
partition | Topic partition id |
fetch_offset | Message offset. |
log_start_offset | Earliest available offset of the follower replica. The field is only used when request is sent by follower. |
max_bytes | Maximum bytes to fetch. |
forgetten_topics_data | Topics to remove from the fetch session. |
topic | Name of topic |
partitions | Partitions to remove from the fetch session. |
Responses:
Fetch Response (Version: 0) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark
partition => INT32
error_code => INT16
high_watermark => INT64
record_set => RECORDS
Field |
Description |
---|
responses | null |
topic | Name of topic |
partition_responses | null |
partition_header | null |
partition | Topic partition id |
error_code | Response error code |
high_watermark | Last committed offset. |
record_set | null |
Fetch Response (Version: 1) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark
partition => INT32
error_code => INT16
high_watermark => INT64
record_set => RECORDS
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
responses | null |
topic | Name of topic |
partition_responses | null |
partition_header | null |
partition | Topic partition id |
error_code | Response error code |
high_watermark | Last committed offset. |
record_set | null |
Fetch Response (Version: 2) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark
partition => INT32
error_code => INT16
high_watermark => INT64
record_set => RECORDS
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
responses | null |
topic | Name of topic |
partition_responses | null |
partition_header | null |
partition | Topic partition id |
error_code | Response error code |
high_watermark | Last committed offset. |
record_set | null |
Fetch Response (Version: 3) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark
partition => INT32
error_code => INT16
high_watermark => INT64
record_set => RECORDS
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
responses | null |
topic | Name of topic |
partition_responses | null |
partition_header | null |
partition | Topic partition id |
error_code | Response error code |
high_watermark | Last committed offset. |
record_set | null |
Fetch Response (Version: 4) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark last_stable_offset [aborted_transactions]
partition => INT32
error_code => INT16
high_watermark => INT64
last_stable_offset => INT64
aborted_transactions => producer_id first_offset
producer_id => INT64
first_offset => INT64
record_set => RECORDS
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
responses | null |
topic | Name of topic |
partition_responses | null |
partition_header | null |
partition | Topic partition id |
error_code | Response error code |
high_watermark | Last committed offset. |
last_stable_offset | The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED) |
aborted_transactions | null |
producer_id | The producer id associated with the aborted transactions |
first_offset | The first offset in the aborted transaction |
record_set | null |
Fetch Response (Version: 5) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions]
partition => INT32
error_code => INT16
high_watermark => INT64
last_stable_offset => INT64
log_start_offset => INT64
aborted_transactions => producer_id first_offset
producer_id => INT64
first_offset => INT64
record_set => RECORDS
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
responses | null |
topic | Name of topic |
partition_responses | null |
partition_header | null |
partition | Topic partition id |
error_code | Response error code |
high_watermark | Last committed offset. |
last_stable_offset | The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED) |
log_start_offset | Earliest available offset. |
aborted_transactions | null |
producer_id | The producer id associated with the aborted transactions |
first_offset | The first offset in the aborted transaction |
record_set | null |
Fetch Response (Version: 6) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions]
partition => INT32
error_code => INT16
high_watermark => INT64
last_stable_offset => INT64
log_start_offset => INT64
aborted_transactions => producer_id first_offset
producer_id => INT64
first_offset => INT64
record_set => RECORDS
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
responses | null |
topic | Name of topic |
partition_responses | null |
partition_header | null |
partition | Topic partition id |
error_code | Response error code |
high_watermark | Last committed offset. |
last_stable_offset | The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED) |
log_start_offset | Earliest available offset. |
aborted_transactions | null |
producer_id | The producer id associated with the aborted transactions |
first_offset | The first offset in the aborted transaction |
record_set | null |
Fetch Response (Version: 7) => throttle_time_ms error_code session_id [responses]
throttle_time_ms => INT32
error_code => INT16
session_id => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions]
partition => INT32
error_code => INT16
high_watermark => INT64
last_stable_offset => INT64
log_start_offset => INT64
aborted_transactions => producer_id first_offset
producer_id => INT64
first_offset => INT64
record_set => RECORDS
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
error_code | Response error code |
session_id | The fetch session ID |
responses | null |
topic | Name of topic |
partition_responses | null |
partition_header | null |
partition | Topic partition id |
error_code | Response error code |
high_watermark | Last committed offset. |
last_stable_offset | The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED) |
log_start_offset | Earliest available offset. |
aborted_transactions | null |
producer_id | The producer id associated with the aborted transactions |
first_offset | The first offset in the aborted transaction |
record_set | null |
Requests:
ListOffsets Request (Version: 0) => replica_id [topics]
replica_id => INT32
topics => topic [partitions]
topic => STRING
partitions => partition timestamp max_num_offsets
partition => INT32
timestamp => INT64
max_num_offsets => INT32
Field |
Description |
---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
topics | Topics to list offsets. |
topic | Name of topic |
partitions | Partitions to list offset. |
partition | Topic partition id |
timestamp | Timestamp. |
max_num_offsets | Maximum offsets to return. |
ListOffsets Request (Version: 1) => replica_id [topics]
replica_id => INT32
topics => topic [partitions]
topic => STRING
partitions => partition timestamp
partition => INT32
timestamp => INT64
Field |
Description |
---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
topics | Topics to list offsets. |
topic | Name of topic |
partitions | Partitions to list offset. |
partition | Topic partition id |
timestamp | The target timestamp for the partition. |
ListOffsets Request (Version: 2) => replica_id isolation_level [topics]
replica_id => INT32
isolation_level => INT8
topics => topic [partitions]
topic => STRING
partitions => partition timestamp
partition => INT32
timestamp => INT64
Field |
Description |
---|
replica_id | Broker id of the follower. For normal consumers, use -1. |
isolation_level | This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records |
topics | Topics to list offsets. |
topic | Name of topic |
partitions | Partitions to list offset. |
partition | Topic partition id |
timestamp | The target timestamp for the partition. |
Responses:
ListOffsets Response (Version: 0) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code [offsets]
partition => INT32
error_code => INT16
offsets => INT64
Field |
Description |
---|
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
error_code | Response error code |
offsets | A list of offsets. |
ListOffsets Response (Version: 1) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code timestamp offset
partition => INT32
error_code => INT16
timestamp => INT64
offset => INT64
Field |
Description |
---|
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
error_code | Response error code |
timestamp | The timestamp associated with the returned offset |
offset | offset found |
ListOffsets Response (Version: 2) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code timestamp offset
partition => INT32
error_code => INT16
timestamp => INT64
offset => INT64
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
error_code | Response error code |
timestamp | The timestamp associated with the returned offset |
offset | offset found |
Requests:
Metadata Request (Version: 0) => [topics]
topics => STRING
Field |
Description |
---|
topics | An array of topics to fetch metadata for. If no topics are specified fetch metadata for all topics. |
Metadata Request (Version: 1) => [topics]
topics => STRING
Field |
Description |
---|
topics | An array of topics to fetch metadata for. If the topics array is null fetch metadata for all topics. |
Metadata Request (Version: 2) => [topics]
topics => STRING
Field |
Description |
---|
topics | An array of topics to fetch metadata for. If the topics array is null fetch metadata for all topics. |
Metadata Request (Version: 3) => [topics]
topics => STRING
Field |
Description |
---|
topics | An array of topics to fetch metadata for. If the topics array is null fetch metadata for all topics. |
Metadata Request (Version: 4) => [topics] allow_auto_topic_creation
topics => STRING
allow_auto_topic_creation => BOOLEAN
Field |
Description |
---|
topics | An array of topics to fetch metadata for. If the topics array is null fetch metadata for all topics. |
allow_auto_topic_creation | If this and the broker config 'auto.create.topics.enable' are true, topics that don't exist will be created by the broker. Otherwise, no topics will be created by the broker. |
Metadata Request (Version: 5) => [topics] allow_auto_topic_creation
topics => STRING
allow_auto_topic_creation => BOOLEAN
Field |
Description |
---|
topics | An array of topics to fetch metadata for. If the topics array is null fetch metadata for all topics. |
allow_auto_topic_creation | If this and the broker config 'auto.create.topics.enable' are true, topics that don't exist will be created by the broker. Otherwise, no topics will be created by the broker. |
Responses:
Metadata Response (Version: 0) => [brokers] [topic_metadata]
brokers => node_id host port
node_id => INT32
host => STRING
port => INT32
topic_metadata => error_code topic [partition_metadata]
error_code => INT16
topic => STRING
partition_metadata => error_code partition leader [replicas] [isr]
error_code => INT16
partition => INT32
leader => INT32
replicas => INT32
isr => INT32
Field |
Description |
---|
brokers | Host and port information for all brokers. |
node_id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
topic_metadata | null |
error_code | Response error code |
topic | Name of topic |
partition_metadata | Metadata for each partition of the topic. |
error_code | Response error code |
partition | Topic partition id |
leader | The id of the broker acting as leader for this partition. |
replicas | The set of all nodes that host this partition. |
isr | The set of nodes that are in sync with the leader for this partition. |
Metadata Response (Version: 1) => [brokers] controller_id [topic_metadata]
brokers => node_id host port rack
node_id => INT32
host => STRING
port => INT32
rack => NULLABLE_STRING
controller_id => INT32
topic_metadata => error_code topic is_internal [partition_metadata]
error_code => INT16
topic => STRING
is_internal => BOOLEAN
partition_metadata => error_code partition leader [replicas] [isr]
error_code => INT16
partition => INT32
leader => INT32
replicas => INT32
isr => INT32
Field |
Description |
---|
brokers | Host and port information for all brokers. |
node_id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
rack | The rack of the broker. |
controller_id | The broker id of the controller broker. |
topic_metadata | null |
error_code | Response error code |
topic | Name of topic |
is_internal | Indicates if the topic is considered a Kafka internal topic |
partition_metadata | Metadata for each partition of the topic. |
error_code | Response error code |
partition | Topic partition id |
leader | The id of the broker acting as leader for this partition. |
replicas | The set of all nodes that host this partition. |
isr | The set of nodes that are in sync with the leader for this partition. |
Metadata Response (Version: 2) => [brokers] cluster_id controller_id [topic_metadata]
brokers => node_id host port rack
node_id => INT32
host => STRING
port => INT32
rack => NULLABLE_STRING
cluster_id => NULLABLE_STRING
controller_id => INT32
topic_metadata => error_code topic is_internal [partition_metadata]
error_code => INT16
topic => STRING
is_internal => BOOLEAN
partition_metadata => error_code partition leader [replicas] [isr]
error_code => INT16
partition => INT32
leader => INT32
replicas => INT32
isr => INT32
Field |
Description |
---|
brokers | Host and port information for all brokers. |
node_id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
rack | The rack of the broker. |
cluster_id | The cluster id that this broker belongs to. |
controller_id | The broker id of the controller broker. |
topic_metadata | null |
error_code | Response error code |
topic | Name of topic |
is_internal | Indicates if the topic is considered a Kafka internal topic |
partition_metadata | Metadata for each partition of the topic. |
error_code | Response error code |
partition | Topic partition id |
leader | The id of the broker acting as leader for this partition. |
replicas | The set of all nodes that host this partition. |
isr | The set of nodes that are in sync with the leader for this partition. |
Metadata Response (Version: 3) => throttle_time_ms [brokers] cluster_id controller_id [topic_metadata]
throttle_time_ms => INT32
brokers => node_id host port rack
node_id => INT32
host => STRING
port => INT32
rack => NULLABLE_STRING
cluster_id => NULLABLE_STRING
controller_id => INT32
topic_metadata => error_code topic is_internal [partition_metadata]
error_code => INT16
topic => STRING
is_internal => BOOLEAN
partition_metadata => error_code partition leader [replicas] [isr]
error_code => INT16
partition => INT32
leader => INT32
replicas => INT32
isr => INT32
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
brokers | Host and port information for all brokers. |
node_id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
rack | The rack of the broker. |
cluster_id | The cluster id that this broker belongs to. |
controller_id | The broker id of the controller broker. |
topic_metadata | null |
error_code | Response error code |
topic | Name of topic |
is_internal | Indicates if the topic is considered a Kafka internal topic |
partition_metadata | Metadata for each partition of the topic. |
error_code | Response error code |
partition | Topic partition id |
leader | The id of the broker acting as leader for this partition. |
replicas | The set of all nodes that host this partition. |
isr | The set of nodes that are in sync with the leader for this partition. |
Metadata Response (Version: 4) => throttle_time_ms [brokers] cluster_id controller_id [topic_metadata]
throttle_time_ms => INT32
brokers => node_id host port rack
node_id => INT32
host => STRING
port => INT32
rack => NULLABLE_STRING
cluster_id => NULLABLE_STRING
controller_id => INT32
topic_metadata => error_code topic is_internal [partition_metadata]
error_code => INT16
topic => STRING
is_internal => BOOLEAN
partition_metadata => error_code partition leader [replicas] [isr]
error_code => INT16
partition => INT32
leader => INT32
replicas => INT32
isr => INT32
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
brokers | Host and port information for all brokers. |
node_id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
rack | The rack of the broker. |
cluster_id | The cluster id that this broker belongs to. |
controller_id | The broker id of the controller broker. |
topic_metadata | null |
error_code | Response error code |
topic | Name of topic |
is_internal | Indicates if the topic is considered a Kafka internal topic |
partition_metadata | Metadata for each partition of the topic. |
error_code | Response error code |
partition | Topic partition id |
leader | The id of the broker acting as leader for this partition. |
replicas | The set of all nodes that host this partition. |
isr | The set of nodes that are in sync with the leader for this partition. |
Metadata Response (Version: 5) => throttle_time_ms [brokers] cluster_id controller_id [topic_metadata]
throttle_time_ms => INT32
brokers => node_id host port rack
node_id => INT32
host => STRING
port => INT32
rack => NULLABLE_STRING
cluster_id => NULLABLE_STRING
controller_id => INT32
topic_metadata => error_code topic is_internal [partition_metadata]
error_code => INT16
topic => STRING
is_internal => BOOLEAN
partition_metadata => error_code partition leader [replicas] [isr] [offline_replicas]
error_code => INT16
partition => INT32
leader => INT32
replicas => INT32
isr => INT32
offline_replicas => INT32
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
brokers | Host and port information for all brokers. |
node_id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
rack | The rack of the broker. |
cluster_id | The cluster id that this broker belongs to. |
controller_id | The broker id of the controller broker. |
topic_metadata | null |
error_code | Response error code |
topic | Name of topic |
is_internal | Indicates if the topic is considered a Kafka internal topic |
partition_metadata | Metadata for each partition of the topic. |
error_code | Response error code |
partition | Topic partition id |
leader | The id of the broker acting as leader for this partition. |
replicas | The set of all nodes that host this partition. |
isr | The set of nodes that are in sync with the leader for this partition. |
offline_replicas | The set of offline replicas of this partition. |
Requests:
LeaderAndIsr Request (Version: 0) => controller_id controller_epoch [partition_states] [live_leaders]
controller_id => INT32
controller_epoch => INT32
partition_states => topic partition controller_epoch leader leader_epoch [isr] zk_version [replicas]
topic => STRING
partition => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
live_leaders => id host port
id => INT32
host => STRING
port => INT32
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
partition_states | null |
topic | Name of topic |
partition | Topic partition id |
controller_epoch | The controller epoch. |
leader | The broker id for the leader. |
leader_epoch | The leader epoch. |
isr | The in sync replica ids. |
zk_version | The ZK version. |
replicas | The replica ids. |
live_leaders | null |
id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
LeaderAndIsr Request (Version: 1) => controller_id controller_epoch [partition_states] [live_leaders]
controller_id => INT32
controller_epoch => INT32
partition_states => topic partition controller_epoch leader leader_epoch [isr] zk_version [replicas] is_new
topic => STRING
partition => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
is_new => BOOLEAN
live_leaders => id host port
id => INT32
host => STRING
port => INT32
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
partition_states | null |
topic | Name of topic |
partition | Topic partition id |
controller_epoch | The controller epoch. |
leader | The broker id for the leader. |
leader_epoch | The leader epoch. |
isr | The in sync replica ids. |
zk_version | The ZK version. |
replicas | The replica ids. |
is_new | Whether the replica should have existed on the broker or not |
live_leaders | null |
id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
Responses:
LeaderAndIsr Response (Version: 0) => error_code [partitions]
error_code => INT16
partitions => topic partition error_code
topic => STRING
partition => INT32
error_code => INT16
Field |
Description |
---|
error_code | Response error code |
partitions | null |
topic | Name of topic |
partition | Topic partition id |
error_code | Response error code |
LeaderAndIsr Response (Version: 1) => error_code [partitions]
error_code => INT16
partitions => topic partition error_code
topic => STRING
partition => INT32
error_code => INT16
Field |
Description |
---|
error_code | Response error code |
partitions | null |
topic | Name of topic |
partition | Topic partition id |
error_code | Response error code |
Requests:
StopReplica Request (Version: 0) => controller_id controller_epoch delete_partitions [partitions]
controller_id => INT32
controller_epoch => INT32
delete_partitions => BOOLEAN
partitions => topic partition
topic => STRING
partition => INT32
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
delete_partitions | Boolean which indicates if replica's partitions must be deleted. |
partitions | null |
topic | Name of topic |
partition | Topic partition id |
Responses:
StopReplica Response (Version: 0) => error_code [partitions]
error_code => INT16
partitions => topic partition error_code
topic => STRING
partition => INT32
error_code => INT16
Field |
Description |
---|
error_code | Response error code |
partitions | null |
topic | Name of topic |
partition | Topic partition id |
error_code | Response error code |
Requests:
UpdateMetadata Request (Version: 0) => controller_id controller_epoch [partition_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
partition_states => topic partition controller_epoch leader leader_epoch [isr] zk_version [replicas]
topic => STRING
partition => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
live_brokers => id host port
id => INT32
host => STRING
port => INT32
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
partition_states | null |
topic | Name of topic |
partition | Topic partition id |
controller_epoch | The controller epoch. |
leader | The broker id for the leader. |
leader_epoch | The leader epoch. |
isr | The in sync replica ids. |
zk_version | The ZK version. |
replicas | The replica ids. |
live_brokers | null |
id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
UpdateMetadata Request (Version: 1) => controller_id controller_epoch [partition_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
partition_states => topic partition controller_epoch leader leader_epoch [isr] zk_version [replicas]
topic => STRING
partition => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
live_brokers => id [end_points]
id => INT32
end_points => port host security_protocol_type
port => INT32
host => STRING
security_protocol_type => INT16
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
partition_states | null |
topic | Name of topic |
partition | Topic partition id |
controller_epoch | The controller epoch. |
leader | The broker id for the leader. |
leader_epoch | The leader epoch. |
isr | The in sync replica ids. |
zk_version | The ZK version. |
replicas | The replica ids. |
live_brokers | null |
id | The broker id. |
end_points | null |
port | The port on which the broker accepts requests. |
host | The hostname of the broker. |
security_protocol_type | The security protocol type. |
UpdateMetadata Request (Version: 2) => controller_id controller_epoch [partition_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
partition_states => topic partition controller_epoch leader leader_epoch [isr] zk_version [replicas]
topic => STRING
partition => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
live_brokers => id [end_points] rack
id => INT32
end_points => port host security_protocol_type
port => INT32
host => STRING
security_protocol_type => INT16
rack => NULLABLE_STRING
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
partition_states | null |
topic | Name of topic |
partition | Topic partition id |
controller_epoch | The controller epoch. |
leader | The broker id for the leader. |
leader_epoch | The leader epoch. |
isr | The in sync replica ids. |
zk_version | The ZK version. |
replicas | The replica ids. |
live_brokers | null |
id | The broker id. |
end_points | null |
port | The port on which the broker accepts requests. |
host | The hostname of the broker. |
security_protocol_type | The security protocol type. |
rack | The rack |
UpdateMetadata Request (Version: 3) => controller_id controller_epoch [partition_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
partition_states => topic partition controller_epoch leader leader_epoch [isr] zk_version [replicas]
topic => STRING
partition => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
live_brokers => id [end_points] rack
id => INT32
end_points => port host listener_name security_protocol_type
port => INT32
host => STRING
listener_name => STRING
security_protocol_type => INT16
rack => NULLABLE_STRING
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
partition_states | null |
topic | Name of topic |
partition | Topic partition id |
controller_epoch | The controller epoch. |
leader | The broker id for the leader. |
leader_epoch | The leader epoch. |
isr | The in sync replica ids. |
zk_version | The ZK version. |
replicas | The replica ids. |
live_brokers | null |
id | The broker id. |
end_points | null |
port | The port on which the broker accepts requests. |
host | The hostname of the broker. |
listener_name | The listener name. |
security_protocol_type | The security protocol type. |
rack | The rack |
UpdateMetadata Request (Version: 4) => controller_id controller_epoch [partition_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
partition_states => topic partition controller_epoch leader leader_epoch [isr] zk_version [replicas] [offline_replicas]
topic => STRING
partition => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
offline_replicas => INT32
live_brokers => id [end_points] rack
id => INT32
end_points => port host listener_name security_protocol_type
port => INT32
host => STRING
listener_name => STRING
security_protocol_type => INT16
rack => NULLABLE_STRING
Field |
Description |
---|
controller_id | The controller id. |
controller_epoch | The controller epoch. |
partition_states | null |
topic | Name of topic |
partition | Topic partition id |
controller_epoch | The controller epoch. |
leader | The broker id for the leader. |
leader_epoch | The leader epoch. |
isr | The in sync replica ids. |
zk_version | The ZK version. |
replicas | The replica ids. |
offline_replicas | The offline replica ids |
live_brokers | null |
id | The broker id. |
end_points | null |
port | The port on which the broker accepts requests. |
host | The hostname of the broker. |
listener_name | The listener name. |
security_protocol_type | The security protocol type. |
rack | The rack |
Responses:
UpdateMetadata Response (Version: 0) => error_code
error_code => INT16
Field |
Description |
---|
error_code | Response error code |
UpdateMetadata Response (Version: 1) => error_code
error_code => INT16
Field |
Description |
---|
error_code | Response error code |
UpdateMetadata Response (Version: 2) => error_code
error_code => INT16
Field |
Description |
---|
error_code | Response error code |
UpdateMetadata Response (Version: 3) => error_code
error_code => INT16
Field |
Description |
---|
error_code | Response error code |
UpdateMetadata Response (Version: 4) => error_code
error_code => INT16
Field |
Description |
---|
error_code | Response error code |
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. |
Responses:
ControlledShutdown Response (Version: 0) => error_code [partitions_remaining]
error_code => INT16
partitions_remaining => topic partition
topic => STRING
partition => INT32
Field |
Description |
---|
error_code | Response error code |
partitions_remaining | The partitions that the broker still leads. |
topic | Name of topic |
partition | Topic partition id |
ControlledShutdown Response (Version: 1) => error_code [partitions_remaining]
error_code => INT16
partitions_remaining => topic partition
topic => STRING
partition => INT32
Field |
Description |
---|
error_code | Response error code |
partitions_remaining | The partitions that the broker still leads. |
topic | Name of topic |
partition | Topic partition id |
Requests:
OffsetCommit Request (Version: 0) => group_id [topics]
group_id => STRING
topics => topic [partitions]
topic => STRING
partitions => partition offset metadata
partition => INT32
offset => INT64
metadata => NULLABLE_STRING
Field |
Description |
---|
group_id | The unique group identifier |
topics | Topics to commit offsets. |
topic | Name of topic |
partitions | Partitions to commit offsets. |
partition | Topic partition id |
offset | Message offset to be committed. |
metadata | Any associated metadata the client wants to keep. |
OffsetCommit Request (Version: 1) => group_id generation_id member_id [topics]
group_id => STRING
generation_id => INT32
member_id => STRING
topics => topic [partitions]
topic => STRING
partitions => partition offset timestamp metadata
partition => INT32
offset => INT64
timestamp => INT64
metadata => NULLABLE_STRING
Field |
Description |
---|
group_id | The unique group identifier |
generation_id | The generation of the group. |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
topics | Topics to commit offsets. |
topic | Name of topic |
partitions | Partitions to commit offsets. |
partition | Topic partition id |
offset | Message offset to be committed. |
timestamp | Timestamp of the commit |
metadata | Any associated metadata the client wants to keep. |
OffsetCommit Request (Version: 2) => group_id generation_id member_id retention_time [topics]
group_id => STRING
generation_id => INT32
member_id => STRING
retention_time => INT64
topics => topic [partitions]
topic => STRING
partitions => partition offset metadata
partition => INT32
offset => INT64
metadata => NULLABLE_STRING
Field |
Description |
---|
group_id | The unique group identifier |
generation_id | The generation of the group. |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
retention_time | Time period in ms to retain the offset. |
topics | Topics to commit offsets. |
topic | Name of topic |
partitions | Partitions to commit offsets. |
partition | Topic partition id |
offset | Message offset to be committed. |
metadata | Any associated metadata the client wants to keep. |
OffsetCommit Request (Version: 3) => group_id generation_id member_id retention_time [topics]
group_id => STRING
generation_id => INT32
member_id => STRING
retention_time => INT64
topics => topic [partitions]
topic => STRING
partitions => partition offset metadata
partition => INT32
offset => INT64
metadata => NULLABLE_STRING
Field |
Description |
---|
group_id | The unique group identifier |
generation_id | The generation of the group. |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
retention_time | Time period in ms to retain the offset. |
topics | Topics to commit offsets. |
topic | Name of topic |
partitions | Partitions to commit offsets. |
partition | Topic partition id |
offset | Message offset to be committed. |
metadata | Any associated metadata the client wants to keep. |
Responses:
OffsetCommit Response (Version: 0) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code
partition => INT32
error_code => INT16
Field |
Description |
---|
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
error_code | Response error code |
OffsetCommit Response (Version: 1) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code
partition => INT32
error_code => INT16
Field |
Description |
---|
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
error_code | Response error code |
OffsetCommit Response (Version: 2) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code
partition => INT32
error_code => INT16
Field |
Description |
---|
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
error_code | Response error code |
OffsetCommit Response (Version: 3) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code
partition => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
error_code | Response error code |
Requests:
OffsetFetch Request (Version: 0) => group_id [topics]
group_id => STRING
topics => topic [partitions]
topic => STRING
partitions => partition
partition => INT32
Field |
Description |
---|
group_id | The unique group identifier |
topics | Topics to fetch offsets. |
topic | Name of topic |
partitions | Partitions to fetch offsets. |
partition | Topic partition id |
OffsetFetch Request (Version: 1) => group_id [topics]
group_id => STRING
topics => topic [partitions]
topic => STRING
partitions => partition
partition => INT32
Field |
Description |
---|
group_id | The unique group identifier |
topics | Topics to fetch offsets. |
topic | Name of topic |
partitions | Partitions to fetch offsets. |
partition | Topic partition id |
OffsetFetch Request (Version: 2) => group_id [topics]
group_id => STRING
topics => topic [partitions]
topic => STRING
partitions => partition
partition => INT32
Field |
Description |
---|
group_id | The unique group identifier |
topics | Topics to fetch offsets. If the topic array is null fetch offsets for all topics. |
topic | Name of topic |
partitions | Partitions to fetch offsets. |
partition | Topic partition id |
OffsetFetch Request (Version: 3) => group_id [topics]
group_id => STRING
topics => topic [partitions]
topic => STRING
partitions => partition
partition => INT32
Field |
Description |
---|
group_id | The unique group identifier |
topics | Topics to fetch offsets. If the topic array is null fetch offsets for all topics. |
topic | Name of topic |
partitions | Partitions to fetch offsets. |
partition | Topic partition id |
Responses:
OffsetFetch Response (Version: 0) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition offset metadata error_code
partition => INT32
offset => INT64
metadata => NULLABLE_STRING
error_code => INT16
Field |
Description |
---|
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
offset | Last committed message offset. |
metadata | Any associated metadata the client wants to keep. |
error_code | Response error code |
OffsetFetch Response (Version: 1) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition offset metadata error_code
partition => INT32
offset => INT64
metadata => NULLABLE_STRING
error_code => INT16
Field |
Description |
---|
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
offset | Last committed message offset. |
metadata | Any associated metadata the client wants to keep. |
error_code | Response error code |
OffsetFetch Response (Version: 2) => [responses] error_code
responses => topic [partition_responses]
topic => STRING
partition_responses => partition offset metadata error_code
partition => INT32
offset => INT64
metadata => NULLABLE_STRING
error_code => INT16
error_code => INT16
Field |
Description |
---|
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
offset | Last committed message offset. |
metadata | Any associated metadata the client wants to keep. |
error_code | Response error code |
error_code | Response error code |
OffsetFetch Response (Version: 3) => throttle_time_ms [responses] error_code
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition offset metadata error_code
partition => INT32
offset => INT64
metadata => NULLABLE_STRING
error_code => INT16
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
responses | null |
topic | Name of topic |
partition_responses | null |
partition | Topic partition id |
offset | Last committed message offset. |
metadata | Any associated metadata the client wants to keep. |
error_code | Response error code |
error_code | Response error code |
Requests:
FindCoordinator Request (Version: 0) => group_id
group_id => STRING
Field |
Description |
---|
group_id | The unique group identifier |
FindCoordinator Request (Version: 1) => coordinator_key coordinator_type
coordinator_key => STRING
coordinator_type => INT8
Field |
Description |
---|
coordinator_key | Id to use for finding the coordinator (for groups, this is the groupId, for transactional producers, this is the transactional id) |
coordinator_type | The type of coordinator to find (0 = group, 1 = transaction) |
Responses:
FindCoordinator Response (Version: 0) => error_code coordinator
error_code => INT16
coordinator => node_id host port
node_id => INT32
host => STRING
port => INT32
Field |
Description |
---|
error_code | Response error code |
coordinator | Host and port information for the coordinator for a consumer group. |
node_id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
FindCoordinator Response (Version: 1) => throttle_time_ms error_code error_message coordinator
throttle_time_ms => INT32
error_code => INT16
error_message => NULLABLE_STRING
coordinator => node_id host port
node_id => INT32
host => STRING
port => INT32
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
error_code | Response error code |
error_message | Response error message |
coordinator | Host and port information for the coordinator |
node_id | The broker id. |
host | The hostname of the broker. |
port | The port on which the broker accepts requests. |
Requests:
JoinGroup Request (Version: 0) => group_id session_timeout member_id protocol_type [group_protocols]
group_id => STRING
session_timeout => INT32
member_id => STRING
protocol_type => STRING
group_protocols => protocol_name protocol_metadata
protocol_name => STRING
protocol_metadata => BYTES
Field |
Description |
---|
group_id | The unique group identifier |
session_timeout | The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms. |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
protocol_type | Unique name for class of protocols implemented by group |
group_protocols | List of protocols that the member supports |
protocol_name | null |
protocol_metadata | null |
JoinGroup Request (Version: 1) => group_id session_timeout rebalance_timeout member_id protocol_type [group_protocols]
group_id => STRING
session_timeout => INT32
rebalance_timeout => INT32
member_id => STRING
protocol_type => STRING
group_protocols => protocol_name protocol_metadata
protocol_name => STRING
protocol_metadata => BYTES
Field |
Description |
---|
group_id | The unique group identifier |
session_timeout | The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms. |
rebalance_timeout | The maximum time that the coordinator will wait for each member to rejoin when rebalancing the group |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
protocol_type | Unique name for class of protocols implemented by group |
group_protocols | List of protocols that the member supports |
protocol_name | null |
protocol_metadata | null |
JoinGroup Request (Version: 2) => group_id session_timeout rebalance_timeout member_id protocol_type [group_protocols]
group_id => STRING
session_timeout => INT32
rebalance_timeout => INT32
member_id => STRING
protocol_type => STRING
group_protocols => protocol_name protocol_metadata
protocol_name => STRING
protocol_metadata => BYTES
Field |
Description |
---|
group_id | The unique group identifier |
session_timeout | The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms. |
rebalance_timeout | The maximum time that the coordinator will wait for each member to rejoin when rebalancing the group |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
protocol_type | Unique name for class of protocols implemented by group |
group_protocols | List of protocols that the member supports |
protocol_name | null |
protocol_metadata | null |
Responses:
JoinGroup Response (Version: 0) => error_code generation_id group_protocol leader_id member_id [members]
error_code => INT16
generation_id => INT32
group_protocol => STRING
leader_id => STRING
member_id => STRING
members => member_id member_metadata
member_id => STRING
member_metadata => BYTES
Field |
Description |
---|
error_code | Response error code |
generation_id | The generation of the group. |
group_protocol | The group protocol selected by the coordinator |
leader_id | The leader of the group |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
members | null |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
member_metadata | null |
JoinGroup Response (Version: 1) => error_code generation_id group_protocol leader_id member_id [members]
error_code => INT16
generation_id => INT32
group_protocol => STRING
leader_id => STRING
member_id => STRING
members => member_id member_metadata
member_id => STRING
member_metadata => BYTES
Field |
Description |
---|
error_code | Response error code |
generation_id | The generation of the group. |
group_protocol | The group protocol selected by the coordinator |
leader_id | The leader of the group |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
members | null |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
member_metadata | null |
JoinGroup Response (Version: 2) => throttle_time_ms error_code generation_id group_protocol leader_id member_id [members]
throttle_time_ms => INT32
error_code => INT16
generation_id => INT32
group_protocol => STRING
leader_id => STRING
member_id => STRING
members => member_id member_metadata
member_id => STRING
member_metadata => BYTES
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
error_code | Response error code |
generation_id | The generation of the group. |
group_protocol | The group protocol selected by the coordinator |
leader_id | The leader of the group |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
members | null |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
member_metadata | null |
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 unique group identifier |
generation_id | The generation of the group. |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
Heartbeat Request (Version: 1) => group_id generation_id member_id
group_id => STRING
generation_id => INT32
member_id => STRING
Field |
Description |
---|
group_id | The unique group identifier |
generation_id | The generation of the group. |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
Responses:
Heartbeat Response (Version: 0) => error_code
error_code => INT16
Field |
Description |
---|
error_code | Response error code |
Heartbeat Response (Version: 1) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
error_code | Response error code |
Requests:
LeaveGroup Request (Version: 0) => group_id member_id
group_id => STRING
member_id => STRING
Field |
Description |
---|
group_id | The unique group identifier |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
LeaveGroup Request (Version: 1) => group_id member_id
group_id => STRING
member_id => STRING
Field |
Description |
---|
group_id | The unique group identifier |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
Responses:
LeaveGroup Response (Version: 0) => error_code
error_code => INT16
Field |
Description |
---|
error_code | Response error code |
LeaveGroup Response (Version: 1) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
error_code | Response error code |
Requests:
SyncGroup Request (Version: 0) => group_id generation_id member_id [group_assignment]
group_id => STRING
generation_id => INT32
member_id => STRING
group_assignment => member_id member_assignment
member_id => STRING
member_assignment => BYTES
Field |
Description |
---|
group_id | The unique group identifier |
generation_id | The generation of the group. |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
group_assignment | null |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
member_assignment | null |
SyncGroup Request (Version: 1) => group_id generation_id member_id [group_assignment]
group_id => STRING
generation_id => INT32
member_id => STRING
group_assignment => member_id member_assignment
member_id => STRING
member_assignment => BYTES
Field |
Description |
---|
group_id | The unique group identifier |
generation_id | The generation of the group. |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
group_assignment | null |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
member_assignment | null |
Responses:
SyncGroup Response (Version: 0) => error_code member_assignment
error_code => INT16
member_assignment => BYTES
Field |
Description |
---|
error_code | Response error code |
member_assignment | null |
SyncGroup Response (Version: 1) => throttle_time_ms error_code member_assignment
throttle_time_ms => INT32
error_code => INT16
member_assignment => BYTES
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
error_code | Response error code |
member_assignment | null |
Requests:
DescribeGroups Request (Version: 0) => [group_ids]
group_ids => STRING
Field |
Description |
---|
group_ids | List of groupIds to request metadata for (an empty groupId array will return empty group metadata). |
DescribeGroups Request (Version: 1) => [group_ids]
group_ids => STRING
Field |
Description |
---|
group_ids | List of groupIds to request metadata for (an empty groupId array will return empty group metadata). |
Responses:
DescribeGroups Response (Version: 0) => [groups]
groups => error_code group_id state protocol_type protocol [members]
error_code => INT16
group_id => STRING
state => STRING
protocol_type => STRING
protocol => STRING
members => member_id client_id client_host member_metadata member_assignment
member_id => STRING
client_id => STRING
client_host => STRING
member_metadata => BYTES
member_assignment => BYTES
Field |
Description |
---|
groups | null |
error_code | Response error code |
group_id | The unique group identifier |
state | The current state of the group (one of: Dead, Stable, CompletingRebalance, PreparingRebalance, or empty if there is no active group) |
protocol_type | The current group protocol type (will be empty if there is no active group) |
protocol | The current group protocol (only provided if the group is Stable) |
members | Current group members (only provided if the group is not Dead) |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
client_id | The client id used in the member's latest join group request |
client_host | The client host used in the request session corresponding to the member's join group. |
member_metadata | The metadata corresponding to the current group protocol in use (will only be present if the group is stable). |
member_assignment | The current assignment provided by the group leader (will only be present if the group is stable). |
DescribeGroups Response (Version: 1) => throttle_time_ms [groups]
throttle_time_ms => INT32
groups => error_code group_id state protocol_type protocol [members]
error_code => INT16
group_id => STRING
state => STRING
protocol_type => STRING
protocol => STRING
members => member_id client_id client_host member_metadata member_assignment
member_id => STRING
client_id => STRING
client_host => STRING
member_metadata => BYTES
member_assignment => BYTES
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
groups | null |
error_code | Response error code |
group_id | The unique group identifier |
state | The current state of the group (one of: Dead, Stable, CompletingRebalance, PreparingRebalance, or empty if there is no active group) |
protocol_type | The current group protocol type (will be empty if there is no active group) |
protocol | The current group protocol (only provided if the group is Stable) |
members | Current group members (only provided if the group is not Dead) |
member_id | The member id assigned by the group coordinator or null if joining for the first time. |
client_id | The client id used in the member's latest join group request |
client_host | The client host used in the request session corresponding to the member's join group. |
member_metadata | The metadata corresponding to the current group protocol in use (will only be present if the group is stable). |
member_assignment | The current assignment provided by the group leader (will only be present if the group is stable). |
Requests:
ListGroups Request (Version: 0) =>
ListGroups Request (Version: 1) =>
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 | Response error code |
groups | null |
group_id | The unique group identifier |
protocol_type | null |
ListGroups Response (Version: 1) => throttle_time_ms error_code [groups]
throttle_time_ms => INT32
error_code => INT16
groups => group_id protocol_type
group_id => STRING
protocol_type => STRING
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
error_code | Response error code |
groups | null |
group_id | The unique group identifier |
protocol_type | null |
Requests:
SaslHandshake Request (Version: 0) => mechanism
mechanism => STRING
Field |
Description |
---|
mechanism | SASL Mechanism chosen by the client. |
SaslHandshake Request (Version: 1) => mechanism
mechanism => STRING
Field |
Description |
---|
mechanism | SASL Mechanism chosen by the client. |
Responses:
SaslHandshake Response (Version: 0) => error_code [enabled_mechanisms]
error_code => INT16
enabled_mechanisms => STRING
Field |
Description |
---|
error_code | Response error code |
enabled_mechanisms | Array of mechanisms enabled in the server. |
SaslHandshake Response (Version: 1) => error_code [enabled_mechanisms]
error_code => INT16
enabled_mechanisms => STRING
Field |
Description |
---|
error_code | Response error code |
enabled_mechanisms | Array of mechanisms enabled in the server. |
Requests:
ApiVersions Request (Version: 0) =>
ApiVersions Request (Version: 1) =>
Responses:
ApiVersions Response (Version: 0) => error_code [api_versions]
error_code => INT16
api_versions => api_key min_version max_version
api_key => INT16
min_version => INT16
max_version => INT16
Field |
Description |
---|
error_code | Response error code |
api_versions | API versions supported by the broker. |
api_key | API key. |
min_version | Minimum supported version. |
max_version | Maximum supported version. |
ApiVersions Response (Version: 1) => error_code [api_versions] throttle_time_ms
error_code => INT16
api_versions => api_key min_version max_version
api_key => INT16
min_version => INT16
max_version => INT16
throttle_time_ms => INT32
Field |
Description |
---|
error_code | Response error code |
api_versions | API versions supported by the broker. |
api_key | API key. |
min_version | Minimum supported version. |
max_version | Maximum supported version. |
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
Requests:
CreateTopics Request (Version: 0) => [create_topic_requests] timeout
create_topic_requests => topic num_partitions replication_factor [replica_assignment] [config_entries]
topic => STRING
num_partitions => INT32
replication_factor => INT16
replica_assignment => partition [replicas]
partition => INT32
replicas => INT32
config_entries => config_name config_value
config_name => STRING
config_value => NULLABLE_STRING
timeout => INT32
Field |
Description |
---|
create_topic_requests | An array of single topic creation requests. Can not have multiple entries for the same topic. |
topic | Name of topic |
num_partitions | Number of partitions to be created. -1 indicates unset. |
replication_factor | Replication factor for the topic. -1 indicates unset. |
replica_assignment | Replica assignment among kafka brokers for this topic partitions. If this is set num_partitions and replication_factor must be unset. |
partition | Topic partition id |
replicas | The set of all nodes that should host this partition. The first replica in the list is the preferred leader. |
config_entries | Topic level configuration for topic to be set. |
config_name | Configuration name |
config_value | Configuration value |
timeout | The time in ms to wait for a topic to be completely created on the controller node. Values <= 0 will trigger topic creation and return immediately |
CreateTopics Request (Version: 1) => [create_topic_requests] timeout validate_only
create_topic_requests => topic num_partitions replication_factor [replica_assignment] [config_entries]
topic => STRING
num_partitions => INT32
replication_factor => INT16
replica_assignment => partition [replicas]
partition => INT32
replicas => INT32
config_entries => config_name config_value
config_name => STRING
config_value => NULLABLE_STRING
timeout => INT32
validate_only => BOOLEAN
Field |
Description |
---|
create_topic_requests | An array of single topic creation requests. Can not have multiple entries for the same topic. |
topic | Name of topic |
num_partitions | Number of partitions to be created. -1 indicates unset. |
replication_factor | Replication factor for the topic. -1 indicates unset. |
replica_assignment | Replica assignment among kafka brokers for this topic partitions. If this is set num_partitions and replication_factor must be unset. |
partition | Topic partition id |
replicas | The set of all nodes that should host this partition. The first replica in the list is the preferred leader. |
config_entries | Topic level configuration for topic to be set. |
config_name | Configuration name |
config_value | Configuration value |
timeout | The time in ms to wait for a topic to be completely created on the controller node. Values <= 0 will trigger topic creation and return immediately |
validate_only | If this is true, the request will be validated, but the topic won't be created. |
CreateTopics Request (Version: 2) => [create_topic_requests] timeout validate_only
create_topic_requests => topic num_partitions replication_factor [replica_assignment] [config_entries]
topic => STRING
num_partitions => INT32
replication_factor => INT16
replica_assignment => partition [replicas]
partition => INT32
replicas => INT32
config_entries => config_name config_value
config_name => STRING
config_value => NULLABLE_STRING
timeout => INT32
validate_only => BOOLEAN
Field |
Description |
---|
create_topic_requests | An array of single topic creation requests. Can not have multiple entries for the same topic. |
topic | Name of topic |
num_partitions | Number of partitions to be created. -1 indicates unset. |
replication_factor | Replication factor for the topic. -1 indicates unset. |
replica_assignment | Replica assignment among kafka brokers for this topic partitions. If this is set num_partitions and replication_factor must be unset. |
partition | Topic partition id |
replicas | The set of all nodes that should host this partition. The first replica in the list is the preferred leader. |
config_entries | Topic level configuration for topic to be set. |
config_name | Configuration name |
config_value | Configuration value |
timeout | The time in ms to wait for a topic to be completely created on the controller node. Values <= 0 will trigger topic creation and return immediately |
validate_only | If this is true, the request will be validated, but the topic won't be created. |
Responses:
CreateTopics Response (Version: 0) => [topic_errors]
topic_errors => topic error_code
topic => STRING
error_code => INT16
Field |
Description |
---|
topic_errors | An array of per topic error codes. |
topic | Name of topic |
error_code | Response error code |
CreateTopics Response (Version: 1) => [topic_errors]
topic_errors => topic error_code error_message
topic => STRING
error_code => INT16
error_message => NULLABLE_STRING
Field |
Description |
---|
topic_errors | An array of per topic errors. |
topic | Name of topic |
error_code | Response error code |
error_message | Response error message |
CreateTopics Response (Version: 2) => throttle_time_ms [topic_errors]
throttle_time_ms => INT32
topic_errors => topic error_code error_message
topic => STRING
error_code => INT16
error_message => NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
topic_errors | An array of per topic errors. |
topic | Name of topic |
error_code | Response error code |
error_message | Response error message |
Requests:
DeleteTopics Request (Version: 0) => [topics] timeout
topics => STRING
timeout => INT32
Field |
Description |
---|
topics | An array of topics to be deleted. |
timeout | The time in ms to wait for a topic to be completely deleted on the controller node. Values <= 0 will trigger topic deletion and return immediately |
DeleteTopics Request (Version: 1) => [topics] timeout
topics => STRING
timeout => INT32
Field |
Description |
---|
topics | An array of topics to be deleted. |
timeout | The time in ms to wait for a topic to be completely deleted on the controller node. Values <= 0 will trigger topic deletion and return immediately |
Responses:
DeleteTopics Response (Version: 0) => [topic_error_codes]
topic_error_codes => topic error_code
topic => STRING
error_code => INT16
Field |
Description |
---|
topic_error_codes | An array of per topic error codes. |
topic | Name of topic |
error_code | Response error code |
DeleteTopics Response (Version: 1) => throttle_time_ms [topic_error_codes]
throttle_time_ms => INT32
topic_error_codes => topic error_code
topic => STRING
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
topic_error_codes | An array of per topic error codes. |
topic | Name of topic |
error_code | Response error code |
Requests:
DeleteRecords Request (Version: 0) => [topics] timeout
topics => topic [partitions]
topic => STRING
partitions => partition offset
partition => INT32
offset => INT64
timeout => INT32
Field |
Description |
---|
topics | null |
topic | Name of topic |
partitions | null |
partition | Topic partition id |
offset | The offset before which the messages will be deleted. -1 means high-watermark for the partition. |
timeout | The maximum time to await a response in ms. |
Responses:
DeleteRecords Response (Version: 0) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => topic [partitions]
topic => STRING
partitions => partition low_watermark error_code
partition => INT32
low_watermark => INT64
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
topics | null |
topic | Name of topic |
partitions | null |
partition | Topic partition id |
low_watermark | Smallest available offset of all live replicas |
error_code | Response error code |
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 for before aborting idle transactions sent by this producer. |
Responses:
InitProducerId Response (Version: 0) => throttle_time_ms error_code producer_id producer_epoch
throttle_time_ms => INT32
error_code => INT16
producer_id => INT64
producer_epoch => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
error_code | Response error code |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
Requests:
OffsetForLeaderEpoch Request (Version: 0) => [topics]
topics => topic [partitions]
topic => STRING
partitions => partition leader_epoch
partition => INT32
leader_epoch => INT32
Field |
Description |
---|
topics | An array of topics to get epochs for |
topic | Name of topic |
partitions | null |
partition | Topic partition id |
leader_epoch | The epoch |
Responses:
OffsetForLeaderEpoch Response (Version: 0) => [topics]
topics => topic [partitions]
topic => STRING
partitions => error_code partition end_offset
error_code => INT16
partition => INT32
end_offset => INT64
Field |
Description |
---|
topics | An array of topics for which we have leader offsets for some requested Partition Leader Epoch |
topic | Name of topic |
partitions | null |
error_code | Response error code |
partition | Topic partition id |
end_offset | The end offset |
Requests:
AddPartitionsToTxn Request (Version: 0) => transactional_id producer_id producer_epoch [topics]
transactional_id => STRING
producer_id => INT64
producer_epoch => INT16
topics => topic [partitions]
topic => STRING
partitions => INT32
Field |
Description |
---|
transactional_id | The transactional id corresponding to the transaction. |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
topics | The partitions to add to the transaction. |
topic | Name of topic |
partitions | null |
Responses:
AddPartitionsToTxn Response (Version: 0) => throttle_time_ms [errors]
throttle_time_ms => INT32
errors => topic [partition_errors]
topic => STRING
partition_errors => partition error_code
partition => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
errors | null |
topic | Name of topic |
partition_errors | null |
partition | Topic partition id |
error_code | Response error code |
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 |
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 quota violation (Zero if the request did not violate any quota) |
error_code | Response error code |
Requests:
EndTxn Request (Version: 0) => transactional_id producer_id producer_epoch transaction_result
transactional_id => STRING
producer_id => INT64
producer_epoch => INT16
transaction_result => BOOLEAN
Field |
Description |
---|
transactional_id | The transactional id corresponding to the transaction. |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
transaction_result | The result of the transaction (0 = ABORT, 1 = COMMIT) |
Responses:
EndTxn Response (Version: 0) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
error_code | Response error code |
Requests:
WriteTxnMarkers Request (Version: 0) => [transaction_markers]
transaction_markers => producer_id producer_epoch transaction_result [topics] coordinator_epoch
producer_id => INT64
producer_epoch => INT16
transaction_result => BOOLEAN
topics => topic [partitions]
topic => STRING
partitions => INT32
coordinator_epoch => INT32
Field |
Description |
---|
transaction_markers | The transaction markers to be written. |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
transaction_result | The result of the transaction to write to the partitions (false = ABORT, true = COMMIT). |
topics | The partitions to write markers for. |
topic | Name of topic |
partitions | null |
coordinator_epoch | Epoch associated with the transaction state partition hosted by this transaction coordinator |
Responses:
WriteTxnMarkers Response (Version: 0) => [transaction_markers]
transaction_markers => producer_id [topics]
producer_id => INT64
topics => topic [partitions]
topic => STRING
partitions => partition error_code
partition => INT32
error_code => INT16
Field |
Description |
---|
transaction_markers | Errors per partition from writing markers. |
producer_id | Current producer id in use by the transactional id. |
topics | Errors per partition from writing markers. |
topic | Name of topic |
partitions | null |
partition | Topic partition id |
error_code | Response error code |
Requests:
TxnOffsetCommit Request (Version: 0) => transactional_id group_id producer_id producer_epoch [topics]
transactional_id => STRING
group_id => STRING
producer_id => INT64
producer_epoch => INT16
topics => topic [partitions]
topic => STRING
partitions => partition offset metadata
partition => INT32
offset => INT64
metadata => NULLABLE_STRING
Field |
Description |
---|
transactional_id | The transactional id corresponding to the transaction. |
group_id | The unique group identifier |
producer_id | Current producer id in use by the transactional id. |
producer_epoch | Current epoch associated with the producer id. |
topics | The partitions to write markers for. |
topic | Name of topic |
partitions | null |
partition | Topic partition id |
offset | null |
metadata | null |
Responses:
TxnOffsetCommit Response (Version: 0) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => topic [partitions]
topic => STRING
partitions => partition error_code
partition => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
topics | Errors per partition from writing markers. |
topic | Name of topic |
partitions | null |
partition | Topic partition id |
error_code | Response error code |
Requests:
DescribeAcls Request (Version: 0) => resource_type resource_name principal host operation permission_type
resource_type => INT8
resource_name => NULLABLE_STRING
principal => NULLABLE_STRING
host => NULLABLE_STRING
operation => INT8
permission_type => INT8
Field |
Description |
---|
resource_type | The resource type |
resource_name | The resource name filter |
principal | The ACL principal filter |
host | The ACL host filter |
operation | The ACL operation |
permission_type | The ACL permission type |
Responses:
DescribeAcls Response (Version: 0) => throttle_time_ms error_code error_message [resources]
throttle_time_ms => INT32
error_code => INT16
error_message => NULLABLE_STRING
resources => resource_type resource_name [acls]
resource_type => INT8
resource_name => STRING
acls => principal host operation permission_type
principal => STRING
host => STRING
operation => INT8
permission_type => INT8
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
error_code | Response error code |
error_message | Response error message |
resources | The resources and their associated ACLs. |
resource_type | The resource type |
resource_name | The resource name |
acls | null |
principal | The ACL principal |
host | The ACL host |
operation | The ACL operation |
permission_type | The ACL permission type |
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 | null |
resource_type | The resource type |
resource_name | The resource name |
principal | The ACL principal |
host | The ACL host |
operation | The ACL operation |
permission_type | The ACL permission type |
Responses:
CreateAcls Response (Version: 0) => throttle_time_ms [creation_responses]
throttle_time_ms => INT32
creation_responses => error_code error_message
error_code => INT16
error_message => NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
creation_responses | null |
error_code | Response error code |
error_message | Response error message |
Requests:
DeleteAcls Request (Version: 0) => [filters]
filters => resource_type resource_name principal host operation permission_type
resource_type => INT8
resource_name => NULLABLE_STRING
principal => NULLABLE_STRING
host => NULLABLE_STRING
operation => INT8
permission_type => INT8
Field |
Description |
---|
filters | null |
resource_type | The resource type |
resource_name | The resource name filter |
principal | The ACL principal filter |
host | The ACL host filter |
operation | The ACL operation |
permission_type | The ACL permission type |
Responses:
DeleteAcls Response (Version: 0) => throttle_time_ms [filter_responses]
throttle_time_ms => INT32
filter_responses => error_code error_message [matching_acls]
error_code => INT16
error_message => NULLABLE_STRING
matching_acls => error_code error_message resource_type resource_name principal host operation permission_type
error_code => INT16
error_message => NULLABLE_STRING
resource_type => INT8
resource_name => STRING
principal => STRING
host => STRING
operation => INT8
permission_type => INT8
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
filter_responses | null |
error_code | Response error code |
error_message | Response error message |
matching_acls | The matching ACLs |
error_code | Response error code |
error_message | Response error message |
resource_type | The resource type |
resource_name | The resource name |
principal | The ACL principal |
host | The ACL host |
operation | The ACL operation |
permission_type | The ACL permission type |
Requests:
DescribeConfigs Request (Version: 0) => [resources]
resources => resource_type resource_name [config_names]
resource_type => INT8
resource_name => STRING
config_names => STRING
Field |
Description |
---|
resources | An array of config resources to be returned. |
resource_type | null |
resource_name | null |
config_names | null |
DescribeConfigs Request (Version: 1) => [resources] include_synonyms
resources => resource_type resource_name [config_names]
resource_type => INT8
resource_name => STRING
config_names => STRING
include_synonyms => BOOLEAN
Field |
Description |
---|
resources | An array of config resources to be returned. |
resource_type | null |
resource_name | null |
config_names | null |
include_synonyms | null |
Responses:
DescribeConfigs Response (Version: 0) => throttle_time_ms [resources]
throttle_time_ms => INT32
resources => error_code error_message resource_type resource_name [config_entries]
error_code => INT16
error_message => NULLABLE_STRING
resource_type => INT8
resource_name => STRING
config_entries => config_name config_value read_only is_default is_sensitive
config_name => STRING
config_value => NULLABLE_STRING
read_only => BOOLEAN
is_default => BOOLEAN
is_sensitive => BOOLEAN
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
resources | null |
error_code | Response error code |
error_message | Response error message |
resource_type | null |
resource_name | null |
config_entries | null |
config_name | null |
config_value | null |
read_only | null |
is_default | null |
is_sensitive | null |
DescribeConfigs Response (Version: 1) => throttle_time_ms [resources]
throttle_time_ms => INT32
resources => error_code error_message resource_type resource_name [config_entries]
error_code => INT16
error_message => NULLABLE_STRING
resource_type => INT8
resource_name => STRING
config_entries => config_name config_value read_only config_source is_sensitive [config_synonyms]
config_name => STRING
config_value => NULLABLE_STRING
read_only => BOOLEAN
config_source => INT8
is_sensitive => BOOLEAN
config_synonyms => config_name config_value config_source
config_name => STRING
config_value => NULLABLE_STRING
config_source => INT8
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
resources | null |
error_code | Response error code |
error_message | Response error message |
resource_type | null |
resource_name | null |
config_entries | null |
config_name | null |
config_value | null |
read_only | null |
config_source | null |
is_sensitive | null |
config_synonyms | null |
config_name | null |
config_value | null |
config_source | null |
Requests:
AlterConfigs Request (Version: 0) => [resources] validate_only
resources => resource_type resource_name [config_entries]
resource_type => INT8
resource_name => STRING
config_entries => config_name config_value
config_name => STRING
config_value => NULLABLE_STRING
validate_only => BOOLEAN
Field |
Description |
---|
resources | An array of resources to update with the provided configs. |
resource_type | null |
resource_name | null |
config_entries | null |
config_name | Configuration name |
config_value | Configuration value |
validate_only | null |
Responses:
AlterConfigs Response (Version: 0) => throttle_time_ms [resources]
throttle_time_ms => INT32
resources => error_code error_message resource_type resource_name
error_code => INT16
error_message => NULLABLE_STRING
resource_type => INT8
resource_name => STRING
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
resources | null |
error_code | Response error code |
error_message | Response error message |
resource_type | null |
resource_name | null |
Requests:
AlterReplicaLogDirs Request (Version: 0) => [log_dirs]
log_dirs => log_dir [topics]
log_dir => STRING
topics => topic [partitions]
topic => STRING
partitions => INT32
Field |
Description |
---|
log_dirs | null |
log_dir | The absolute log directory path. |
topics | null |
topic | Name of topic |
partitions | List of partition ids of the topic. |
Responses:
AlterReplicaLogDirs Response (Version: 0) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => topic [partitions]
topic => STRING
partitions => partition error_code
partition => INT32
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
topics | null |
topic | Name of topic |
partitions | null |
partition | Topic partition id |
error_code | Response error code |
Requests:
DescribeLogDirs Request (Version: 0) => [topics]
topics => topic [partitions]
topic => STRING
partitions => INT32
Field |
Description |
---|
topics | null |
topic | Name of topic |
partitions | List of partition ids of the topic. |
Responses:
DescribeLogDirs Response (Version: 0) => throttle_time_ms [log_dirs]
throttle_time_ms => INT32
log_dirs => error_code log_dir [topics]
error_code => INT16
log_dir => STRING
topics => topic [partitions]
topic => STRING
partitions => partition size offset_lag is_future
partition => INT32
size => INT64
offset_lag => INT64
is_future => BOOLEAN
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
log_dirs | null |
error_code | Response error code |
log_dir | The absolute log directory path. |
topics | null |
topic | Name of topic |
partitions | null |
partition | Topic partition id |
size | The size of the log segments of the 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 | True if this log is created by AlterReplicaLogDirsRequest and will replace the current log of the replica in the future. |
Requests:
SaslAuthenticate Request (Version: 0) => sasl_auth_bytes
sasl_auth_bytes => BYTES
Field |
Description |
---|
sasl_auth_bytes | SASL authentication bytes from client as defined by the SASL mechanism. |
Responses:
SaslAuthenticate Response (Version: 0) => error_code error_message sasl_auth_bytes
error_code => INT16
error_message => NULLABLE_STRING
sasl_auth_bytes => BYTES
Field |
Description |
---|
error_code | Response error code |
error_message | Response error message |
sasl_auth_bytes | SASL authentication bytes from server as defined by the SASL mechanism. |
Requests:
CreatePartitions Request (Version: 0) => [topic_partitions] timeout validate_only
topic_partitions => topic new_partitions
topic => STRING
new_partitions => count [assignment]
count => INT32
assignment => ARRAY(INT32)
timeout => INT32
validate_only => BOOLEAN
Field |
Description |
---|
topic_partitions | List of topic and the corresponding new partitions. |
topic | Name of topic |
new_partitions | null |
count | The new partition count. |
assignment | The assigned brokers. |
timeout | 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. |
Responses:
CreatePartitions Response (Version: 0) => throttle_time_ms [topic_errors]
throttle_time_ms => INT32
topic_errors => topic error_code error_message
topic => STRING
error_code => INT16
error_message => NULLABLE_STRING
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
topic_errors | Per topic results for the create partitions request |
topic | Name of topic |
error_code | Response error code |
error_message | Response error message |
Requests:
CreateDelegationToken Request (Version: 0) => [renewers] max_life_time
renewers => principal_type name
principal_type => STRING
name => STRING
max_life_time => INT64
Field |
Description |
---|
renewers | An array of token renewers. Renewer is an Kafka PrincipalType and name string, who is allowed to renew this token before the max lifetime expires. |
principal_type | principalType of the Kafka principal |
name | name of the Kafka principal |
max_life_time | Max lifetime period for token in milli seconds. if value is -1, then max lifetime will default to a server side config value. |
Responses:
CreateDelegationToken Response (Version: 0) => error_code owner issue_timestamp expiry_timestamp max_timestamp token_id hmac throttle_time_ms
error_code => INT16
owner => principal_type name
principal_type => STRING
name => STRING
issue_timestamp => INT64
expiry_timestamp => INT64
max_timestamp => INT64
token_id => STRING
hmac => BYTES
throttle_time_ms => INT32
Field |
Description |
---|
error_code | Response error code |
owner | token owner. |
principal_type | principalType of the Kafka principal |
name | name of the Kafka principal |
issue_timestamp | timestamp (in msec) when this token was generated. |
expiry_timestamp | timestamp (in msec) at which this token expires. |
max_timestamp | max life time of this token. |
token_id | UUID to ensure uniqueness. |
hmac | HMAC of the delegation token. |
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
Requests:
RenewDelegationToken Request (Version: 0) => hmac renew_time_period
hmac => BYTES
renew_time_period => INT64
Field |
Description |
---|
hmac | HMAC of the delegation token to be renewed. |
renew_time_period | Renew time period in milli seconds. |
Responses:
RenewDelegationToken Response (Version: 0) => error_code expiry_timestamp throttle_time_ms
error_code => INT16
expiry_timestamp => INT64
throttle_time_ms => INT32
Field |
Description |
---|
error_code | Response error code |
expiry_timestamp | timestamp (in msec) at which this token expires.. |
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
Requests:
ExpireDelegationToken Request (Version: 0) => hmac expiry_time_period
hmac => BYTES
expiry_time_period => INT64
Field |
Description |
---|
hmac | HMAC of the delegation token to be expired. |
expiry_time_period | expiry time period in milli seconds. |
Responses:
ExpireDelegationToken Response (Version: 0) => error_code expiry_timestamp throttle_time_ms
error_code => INT16
expiry_timestamp => INT64
throttle_time_ms => INT32
Field |
Description |
---|
error_code | Response error code |
expiry_timestamp | timestamp (in msec) at which this token expires.. |
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
Requests:
DescribeDelegationToken Request (Version: 0) => [owners]
owners => principal_type name
principal_type => STRING
name => STRING
Field |
Description |
---|
owners | An array of token owners. |
principal_type | principalType of the Kafka principal |
name | name of the Kafka principal |
Responses:
DescribeDelegationToken Response (Version: 0) => error_code [token_details] throttle_time_ms
error_code => INT16
token_details => owner issue_timestamp expiry_timestamp max_timestamp token_id hmac [renewers]
owner => principal_type name
principal_type => STRING
name => STRING
issue_timestamp => INT64
expiry_timestamp => INT64
max_timestamp => INT64
token_id => STRING
hmac => BYTES
renewers => principal_type name
principal_type => STRING
name => STRING
throttle_time_ms => INT32
Field |
Description |
---|
error_code | Response error code |
token_details | null |
owner | token owner. |
principal_type | principalType of the Kafka principal |
name | name of the Kafka principal |
issue_timestamp | timestamp (in msec) when this token was generated. |
expiry_timestamp | timestamp (in msec) at which this token expires. |
max_timestamp | max life time of this token. |
token_id | UUID to ensure uniqueness. |
hmac | HMAC of the delegation token to be expired. |
renewers | An array of token renewers. Renewer is an Kafka PrincipalType and name string, who is allowed to renew this token before the max lifetime expires. |
principal_type | principalType of the Kafka principal |
name | name of the Kafka principal |
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
Requests:
DeleteGroups Request (Version: 0) => [groups]
groups => STRING
Field |
Description |
---|
groups | An array of groups to be deleted. |
Responses:
DeleteGroups Response (Version: 0) => throttle_time_ms [group_error_codes]
throttle_time_ms => INT32
group_error_codes => group_id error_code
group_id => STRING
error_code => INT16
Field |
Description |
---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota) |
group_error_codes | An array of per group error codes. |
group_id | The unique group identifier |
error_code | Response error code |
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.