diff --git a/40/generated/protocol_messages.html b/40/generated/protocol_messages.html index 6cb593709..e9c322dc9 100644 --- a/40/generated/protocol_messages.html +++ b/40/generated/protocol_messages.html @@ -394,6 +394,43 @@
Produce Request (Version: 13) => transactional_id acks timeout_ms [topic_data] _tagged_fields + transactional_id => COMPACT_NULLABLE_STRING + acks => INT16 + timeout_ms => INT32 + topic_data => topic_id [partition_data] _tagged_fields + topic_id => UUID + partition_data => index records _tagged_fields + index => INT32 + records => COMPACT_RECORDS +
Request header version: 2
+Field | +Description | +
---|---|
transactional_id | The transactional ID, or null if the producer is not transactional. |
acks | The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR. |
timeout_ms | The timeout to await a response in milliseconds. |
topic_data | Each topic to produce to. |
topic_id | The unique topic ID |
partition_data | Each partition to produce to. |
index | The partition index. |
records | The record data to be produced. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Produce Response (Version: 3) => [responses] throttle_time_ms responses => name [partition_responses] @@ -843,6 +880,190 @@Produce API (Key: 0):
Produce Response (Version: 12) => [responses] throttle_time_ms _tagged_fields + responses => name [partition_responses] _tagged_fields + name => COMPACT_STRING + partition_responses => index error_code base_offset log_append_time_ms log_start_offset [record_errors] error_message _tagged_fields + index => INT32 + error_code => INT16 + base_offset => INT64 + log_append_time_ms => INT64 + log_start_offset => INT64 + record_errors => batch_index batch_index_error_message _tagged_fields + batch_index => INT32 + batch_index_error_message => COMPACT_NULLABLE_STRING + error_message => COMPACT_NULLABLE_STRING + throttle_time_ms => INT32 +
Response header version: 1
+Field | +Description | +||||||||||||||||||
---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
responses | Each produce response. | ||||||||||||||||||
name | The topic name. | ||||||||||||||||||
partition_responses | Each partition that we produced to within the topic. | ||||||||||||||||||
index | The partition index. | ||||||||||||||||||
error_code | The error code, or 0 if there was no error. | ||||||||||||||||||
base_offset | The base offset. | ||||||||||||||||||
log_append_time_ms | The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended. | ||||||||||||||||||
log_start_offset | The log start offset. | ||||||||||||||||||
record_errors | The batch indices of records that caused the batch to be dropped. | ||||||||||||||||||
batch_index | The batch index of the record that caused the batch to be dropped. | ||||||||||||||||||
batch_index_error_message | The error message of the record that caused the batch to be dropped. | ||||||||||||||||||
_tagged_fields | The tagged fields | ||||||||||||||||||
error_message | The global error message summarizing the common root cause of the records that caused the batch to be dropped. | ||||||||||||||||||
_tagged_fields |
| ||||||||||||||||||
_tagged_fields | The tagged fields | ||||||||||||||||||
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. | ||||||||||||||||||
_tagged_fields |
|
Produce Response (Version: 13) => [responses] throttle_time_ms _tagged_fields + responses => topic_id [partition_responses] _tagged_fields + topic_id => UUID + partition_responses => index error_code base_offset log_append_time_ms log_start_offset [record_errors] error_message _tagged_fields + index => INT32 + error_code => INT16 + base_offset => INT64 + log_append_time_ms => INT64 + log_start_offset => INT64 + record_errors => batch_index batch_index_error_message _tagged_fields + batch_index => INT32 + batch_index_error_message => COMPACT_NULLABLE_STRING + error_message => COMPACT_NULLABLE_STRING + throttle_time_ms => INT32 +
Response header version: 1
+Field | +Description | +||||||||||||||||||
---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
responses | Each produce response. | ||||||||||||||||||
topic_id | The unique topic ID | ||||||||||||||||||
partition_responses | Each partition that we produced to within the topic. | ||||||||||||||||||
index | The partition index. | ||||||||||||||||||
error_code | The error code, or 0 if there was no error. | ||||||||||||||||||
base_offset | The base offset. | ||||||||||||||||||
log_append_time_ms | The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended. | ||||||||||||||||||
log_start_offset | The log start offset. | ||||||||||||||||||
record_errors | The batch indices of records that caused the batch to be dropped. | ||||||||||||||||||
batch_index | The batch index of the record that caused the batch to be dropped. | ||||||||||||||||||
batch_index_error_message | The error message of the record that caused the batch to be dropped. | ||||||||||||||||||
_tagged_fields | The tagged fields | ||||||||||||||||||
error_message | The global error message summarizing the common root cause of the records that caused the batch to be dropped. | ||||||||||||||||||
_tagged_fields |
| ||||||||||||||||||
_tagged_fields | The tagged fields | ||||||||||||||||||
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. | ||||||||||||||||||
_tagged_fields |
|
ListOffsets Request (Version: 1) => replica_id [topics] - replica_id => INT32 - topics => name [partitions] - name => STRING - partitions => partition_index timestamp +-Fetch Response (Version: 17) => throttle_time_ms error_code session_id [responses] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + session_id => INT32 + responses => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition_index error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] preferred_read_replica records _tagged_fields partition_index => INT32 - timestamp => INT64 -Request header version: 1
+ error_code => INT16 + high_watermark => INT64 + last_stable_offset => INT64 + log_start_offset => INT64 + aborted_transactions => producer_id first_offset _tagged_fields + producer_id => INT64 + first_offset => INT64 + preferred_read_replica => INT32 + records => COMPACT_RECORDS +Response header version: 1
-
Field Description - +replica_id The broker ID of the requester, or -1 if this request is being made by a normal consumer. throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. - +topics Each topic in the request. error_code The top level response error code. - +name The topic name. session_id The fetch session ID, or 0 if this is not part of a fetch session. - +partitions Each partition in the request. responses The response topics. ++ +topic_id The unique topic ID. + partitions The topic partitions. partition_index The partition index. - -timestamp The current timestamp. +ListOffsets Request (Version: 2) => replica_id isolation_level [topics] +error_code The error code, or 0 if there was no fetch error. ++ +high_watermark The current high water mark. + +last_stable_offset The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED). + +log_start_offset The current log start offset. + +aborted_transactions The aborted transactions. + +producer_id The producer id associated with the aborted transaction. + +first_offset The first offset in the aborted transaction. + +_tagged_fields The tagged fields + +preferred_read_replica The preferred read replica for the consumer to use on its next fetch request. + +records The record data. + +_tagged_fields +
+Tag +Tagged field +Description ++ +0 diverging_epoch In case divergence is detected based on the `LastFetchedEpoch` and `FetchOffset` in the request, this field indicates the largest epoch and its end offset such that subsequent records are known to diverge. +
+Field +Description ++ +epoch The largest epoch. + +end_offset The end offset of the epoch. + +_tagged_fields The tagged fields + +1 current_leader The current leader of the partition. +
+Field +Description ++ +leader_id The ID of the current leader or -1 if the leader is unknown. + +leader_epoch The latest known leader epoch. + +_tagged_fields The tagged fields + +2 snapshot_id In the case of fetching an offset less than the LogStartOffset, this is the end offset and epoch that should be used in the FetchSnapshot request. +
+Field +Description ++ +end_offset The end offset of the epoch. + +epoch The largest epoch. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + + +_tagged_fields +
+Tag +Tagged field +Description ++ +0 node_endpoints Endpoints for all current-leaders enumerated in PartitionData, with errors NOT_LEADER_OR_FOLLOWER & FENCED_LEADER_EPOCH. +
+Field +Description ++ +node_id The ID of the associated node. + +host The node's hostname. + +port The node's port. + +rack The rack of the node, or null if it has not been assigned to a rack. + +_tagged_fields The tagged fields ListOffsets API (Key: 2):
+ +Requests:
++ListOffsets Request (Version: 1) => replica_id [topics] + replica_id => INT32 + topics => name [partitions] + name => STRING + partitions => partition_index timestamp + partition_index => INT32 + timestamp => INT64 +Request header version: 1
++
+Field +Description ++ +replica_id The broker ID of the requester, or -1 if this request is being made by a normal consumer. + +topics Each topic in the request. + +name The topic name. + +partitions Each partition in the request. + +partition_index The partition index. + +timestamp The current timestamp. +ListOffsets Request (Version: 2) => replica_id isolation_level [topics] replica_id => INT32 isolation_level => INT8 topics => name [partitions] @@ -3393,6 +3739,46 @@ListOffsets API (Key: 2):
_tagged_fields The tagged fields ListOffsets Response (Version: 10) => throttle_time_ms [topics] _tagged_fields + throttle_time_ms => INT32 + topics => name [partitions] _tagged_fields + name => COMPACT_STRING + partitions => partition_index error_code timestamp offset leader_epoch _tagged_fields + partition_index => INT32 + error_code => INT16 + timestamp => INT64 + offset => INT64 + leader_epoch => INT32 +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +topics Each topic in the response. + +name The topic name. + +partitions Each partition in the response. + +partition_index The partition index. + +error_code The partition error code, or 0 if there was no error. + +timestamp The timestamp associated with the returned offset. + +offset The returned offset. + +leader_epoch The leader epoch associated with the returned offset. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields Metadata API (Key: 3):
Requests:
@@ -4544,6 +4930,90 @@Metadata API (Key: 3):
_tagged_fields The tagged fields
Metadata Response (Version: 13) => throttle_time_ms [brokers] cluster_id controller_id [topics] error_code _tagged_fields + throttle_time_ms => INT32 + brokers => node_id host port rack _tagged_fields + node_id => INT32 + host => COMPACT_STRING + port => INT32 + rack => COMPACT_NULLABLE_STRING + cluster_id => COMPACT_NULLABLE_STRING + controller_id => INT32 + topics => error_code name topic_id is_internal [partitions] topic_authorized_operations _tagged_fields + error_code => INT16 + name => COMPACT_NULLABLE_STRING + topic_id => UUID + is_internal => BOOLEAN + partitions => error_code partition_index leader_id leader_epoch [replica_nodes] [isr_nodes] [offline_replicas] _tagged_fields + error_code => INT16 + partition_index => INT32 + leader_id => INT32 + leader_epoch => INT32 + replica_nodes => INT32 + isr_nodes => INT32 + offline_replicas => INT32 + topic_authorized_operations => INT32 + error_code => INT16 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
brokers | A list of brokers present in the cluster. |
node_id | The broker ID. |
host | The broker hostname. |
port | The broker port. |
rack | The rack of the broker, or null if it has not been assigned to a rack. |
_tagged_fields | The tagged fields |
cluster_id | The cluster ID that responding broker belongs to. |
controller_id | The ID of the controller broker. |
topics | Each topic in the response. |
error_code | The topic error, or 0 if there was no error. |
name | The topic name. Null for non-existing topics queried by ID. This is never null when ErrorCode is zero. One of Name and TopicId is always populated. |
topic_id | The topic id. Zero for non-existing topics queried by name. This is never zero when ErrorCode is zero. One of Name and TopicId is always populated. |
is_internal | True if the topic is internal. |
partitions | Each partition in the topic. |
error_code | The partition error, or 0 if there was no error. |
partition_index | The partition index. |
leader_id | The ID of the leader broker. |
leader_epoch | The leader epoch of this partition. |
replica_nodes | The set of all nodes that host this partition. |
isr_nodes | The set of nodes that are in sync with the leader for this partition. |
offline_replicas | The set of offline replicas of this partition. |
_tagged_fields | The tagged fields |
topic_authorized_operations | 32-bit bitfield to represent authorized operations for this topic. |
_tagged_fields | The tagged fields |
error_code | The top-level error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
OffsetCommit Response (Version: 2) => [topics] - topics => name [partitions] - name => STRING - partitions => partition_index error_code ++OffsetCommit Request (Version: 10) => group_id generation_id_or_member_epoch member_id group_instance_id [topics] _tagged_fields + group_id => COMPACT_STRING + generation_id_or_member_epoch => INT32 + member_id => COMPACT_STRING + group_instance_id => COMPACT_NULLABLE_STRING + topics => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition_index committed_offset committed_leader_epoch committed_metadata _tagged_fields partition_index => INT32 - error_code => INT16 -Response header version: 0
--
Field + committed_offset => INT64 + committed_leader_epoch => INT32 + committed_metadata => COMPACT_NULLABLE_STRING +This version of the request is unstable.
Request header version: 2
++
+ +Responses:Field +Description ++ +group_id The unique group identifier. + +generation_id_or_member_epoch The generation of the group if using the classic group protocol or the member epoch if using the consumer protocol. + +member_id The member ID assigned by the group coordinator. + +group_instance_id The unique identifier of the consumer instance provided by end user. + +topics The topics to commit offsets for. + +topic_id The topic ID. + +partitions Each partition to commit offsets for. + +partition_index The partition index. + +committed_offset The message offset to be committed. + +committed_leader_epoch The leader epoch of this partition. + +committed_metadata Any associated metadata the client wants to keep. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields
++OffsetCommit Response (Version: 2) => [topics] + topics => name [partitions] + name => STRING + partitions => partition_index error_code + partition_index => INT32 + error_code => INT16 +Response header version: 0
++
Field Description @@ -5040,6 +5556,68 @@ topics The responses for each topic. OffsetCommit API (Key: 8):
_tagged_fields The tagged fields +OffsetCommit Response (Version: 9) => throttle_time_ms [topics] _tagged_fields + throttle_time_ms => INT32 + topics => name [partitions] _tagged_fields + name => COMPACT_STRING + partitions => partition_index error_code _tagged_fields + partition_index => INT32 + error_code => INT16 +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +topics The responses for each topic. + +name The topic name. + +partitions The responses for each partition in the topic. + +partition_index The partition index. + +error_code The error code, or 0 if there was no error. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields OffsetCommit Response (Version: 10) => throttle_time_ms [topics] _tagged_fields + throttle_time_ms => INT32 + topics => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition_index error_code _tagged_fields + partition_index => INT32 + error_code => INT16 +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +topics The responses for each topic. + +topic_id The topic ID. + +partitions The responses for each partition in the topic. + +partition_index The partition index. + +error_code The error code, or 0 if there was no error. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields OffsetFetch API (Key: 9):
Requests:
@@ -5255,6 +5833,43 @@OffsetFetch API (Key: 9):
_tagged_fields The tagged fields Responses:OffsetFetch Request (Version: 10) => [groups] require_stable _tagged_fields + groups => group_id member_id member_epoch [topics] _tagged_fields + group_id => COMPACT_STRING + member_id => COMPACT_NULLABLE_STRING + member_epoch => INT32 + topics => topic_id [partition_indexes] _tagged_fields + topic_id => UUID + partition_indexes => INT32 + require_stable => BOOLEAN +This version of the request is unstable.
Request header version: 2
++
+Field +Description ++ +groups Each group we would like to fetch offsets for. + +group_id The group ID. + +member_id The member id. + +member_epoch The member epoch if using the new consumer protocol (KIP-848). + +topics Each topic we would like to fetch offsets for, or null to fetch offsets for all topics. + +topic_id The topic ID. + +partition_indexes The partition indexes we would like to fetch offsets for. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +require_stable Whether broker should hold on returning unstable offsets but set a retriable error code for the partitions. + +_tagged_fields The tagged fields
+OffsetFetch Response (Version: 1) => [topics] topics => name [partitions] @@ -5557,6 +6172,108 @@OffsetFetch API (Key: 9):
_tagged_fields The tagged fields +OffsetFetch Response (Version: 9) => throttle_time_ms [groups] _tagged_fields + throttle_time_ms => INT32 + groups => group_id [topics] error_code _tagged_fields + group_id => COMPACT_STRING + topics => name [partitions] _tagged_fields + name => COMPACT_STRING + partitions => partition_index committed_offset committed_leader_epoch metadata error_code _tagged_fields + partition_index => INT32 + committed_offset => INT64 + committed_leader_epoch => INT32 + metadata => COMPACT_NULLABLE_STRING + error_code => INT16 + error_code => INT16 +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +groups The responses per group id. + +group_id The group ID. + +topics The responses per topic. + +name The topic name. + +partitions The responses per partition. + +partition_index The partition index. + +committed_offset The committed message offset. + +committed_leader_epoch The leader epoch. + +metadata The partition metadata. + +error_code The partition-level error code, or 0 if there was no error. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +error_code The group-level error code, or 0 if there was no error. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields OffsetFetch Response (Version: 10) => throttle_time_ms [groups] _tagged_fields + throttle_time_ms => INT32 + groups => group_id [topics] error_code _tagged_fields + group_id => COMPACT_STRING + topics => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition_index committed_offset committed_leader_epoch metadata error_code _tagged_fields + partition_index => INT32 + committed_offset => INT64 + committed_leader_epoch => INT32 + metadata => COMPACT_NULLABLE_STRING + error_code => INT16 + error_code => INT16 +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +groups The responses per group id. + +group_id The group ID. + +topics The responses per topic. + +topic_id The topic ID. + +partitions The responses per partition. + +partition_index The partition index. + +committed_offset The committed message offset. + +committed_leader_epoch The leader epoch. + +metadata The partition metadata. + +error_code The partition-level error code, or 0 if there was no error. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +error_code The group-level error code, or 0 if there was no error. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields FindCoordinator API (Key: 10):
Requests:
@@ -5823,21 +6540,56 @@FindCoordinator API (Key: 10):_tagged_fields
The tagged fields
JoinGroup Request (Version: 2) => group_id session_timeout_ms rebalance_timeout_ms member_id protocol_type [protocols] - group_id => STRING - session_timeout_ms => INT32 - rebalance_timeout_ms => INT32 - member_id => STRING - protocol_type => STRING - protocols => name metadata - name => STRING - metadata => BYTES -
Request header version: 1
-Field | +
---|
Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
coordinators | Each coordinator result in the response. |
key | The coordinator key. |
node_id | The node id. |
host | The host name. |
port | The port. |
error_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
JoinGroup Request (Version: 2) => group_id session_timeout_ms rebalance_timeout_ms member_id protocol_type [protocols] + group_id => STRING + session_timeout_ms => INT32 + rebalance_timeout_ms => INT32 + member_id => STRING + protocol_type => STRING + protocols => name metadata + name => STRING + metadata => BYTES +
Request header version: 1
+Field | Description |
---|---|
group_id | The group identifier. | _tagged_fields | The tagged fields |
JoinGroup Response (Version: 9) => throttle_time_ms error_code generation_id protocol_type protocol_name leader skip_assignment member_id [members] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + generation_id => INT32 + protocol_type => COMPACT_NULLABLE_STRING + protocol_name => COMPACT_NULLABLE_STRING + leader => COMPACT_STRING + skip_assignment => BOOLEAN + member_id => COMPACT_STRING + members => member_id group_instance_id metadata _tagged_fields + member_id => COMPACT_STRING + group_instance_id => COMPACT_NULLABLE_STRING + metadata => COMPACT_BYTES +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The error code, or 0 if there was no error. |
generation_id | The generation ID of the group. |
protocol_type | The group protocol name. |
protocol_name | The group protocol selected by the coordinator. |
leader | The leader of the group. |
skip_assignment | True if the leader must skip running the assignment. |
member_id | The member ID assigned by the group coordinator. |
members | The group members. |
member_id | The group member ID. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
metadata | The group member metadata. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
Heartbeat Response (Version: 4) => throttle_time_ms error_code _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
LeaveGroup Response (Version: 5) => throttle_time_ms error_code [members] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + members => member_id group_instance_id error_code _tagged_fields + member_id => COMPACT_STRING + group_instance_id => COMPACT_NULLABLE_STRING + error_code => INT16 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The error code, or 0 if there was no error. |
members | List of leaving member responses. |
member_id | The member ID to remove from the group. |
group_instance_id | The group instance ID to remove from the group. |
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
SyncGroup Response (Version: 5) => throttle_time_ms error_code protocol_type protocol_name assignment _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + protocol_type => COMPACT_NULLABLE_STRING + protocol_name => COMPACT_NULLABLE_STRING + assignment => COMPACT_BYTES +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The error code, or 0 if there was no error. |
protocol_type | The group protocol type. |
protocol_name | The group protocol name. |
assignment | The member assignment. |
_tagged_fields | The tagged fields |
DescribeGroups Response (Version: 6) => throttle_time_ms [groups] _tagged_fields + throttle_time_ms => INT32 + groups => error_code error_message group_id group_state protocol_type protocol_data [members] authorized_operations _tagged_fields + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + group_id => COMPACT_STRING + group_state => COMPACT_STRING + protocol_type => COMPACT_STRING + protocol_data => COMPACT_STRING + members => member_id group_instance_id client_id client_host member_metadata member_assignment _tagged_fields + member_id => COMPACT_STRING + group_instance_id => COMPACT_NULLABLE_STRING + client_id => COMPACT_STRING + client_host => COMPACT_STRING + member_metadata => COMPACT_BYTES + member_assignment => COMPACT_BYTES + authorized_operations => INT32 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
groups | Each described group. |
error_code | The describe error, or 0 if there was no error. |
error_message | The describe error message, or null if there was no error. |
group_id | The group ID string. |
group_state | The group state string, or the empty string. |
protocol_type | The group protocol type, or the empty string. |
protocol_data | The group protocol data, or the empty string. |
members | The group members. |
member_id | The member id. |
group_instance_id | The unique identifier of the consumer instance provided by end user. |
client_id | The client ID used in the member's latest join group request. |
client_host | The client host. |
member_metadata | The metadata corresponding to the current group protocol in use. |
member_assignment | The current assignment provided by the group leader. |
_tagged_fields | The tagged fields |
authorized_operations | 32-bit bitfield to represent authorized operations for this group. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
ListGroups Response (Version: 5) => throttle_time_ms error_code [groups] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + groups => group_id protocol_type group_state group_type _tagged_fields + group_id => COMPACT_STRING + protocol_type => COMPACT_STRING + group_state => COMPACT_STRING + group_type => COMPACT_STRING +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The error code, or 0 if there was no error. |
groups | Each group in the response. |
group_id | The group ID. |
protocol_type | The group protocol type. |
group_state | The group state name. |
group_type | The group type name. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
SaslHandshake Response (Version: 1) => error_code [mechanisms] + error_code => INT16 + mechanisms => STRING +
Response header version: 0
+Field | +Description | +
---|---|
error_code | The error code, or 0 if there was no error. |
mechanisms | The mechanisms enabled in the server. |
ApiVersions Response (Version: 4) => error_code [api_keys] throttle_time_ms _tagged_fields + error_code => INT16 + api_keys => api_key min_version max_version _tagged_fields + api_key => INT16 + min_version => INT16 + max_version => INT16 + throttle_time_ms => INT32 +
Response header version: 0
+Field | +Description | +|||||||||||||||||||||||||||||||||||
---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
error_code | The top-level error code. | |||||||||||||||||||||||||||||||||||
api_keys | The APIs supported by the broker. | |||||||||||||||||||||||||||||||||||
api_key | The API index. | |||||||||||||||||||||||||||||||||||
min_version | The minimum supported version, inclusive. | |||||||||||||||||||||||||||||||||||
max_version | The maximum supported version, inclusive. | |||||||||||||||||||||||||||||||||||
_tagged_fields | The tagged fields | |||||||||||||||||||||||||||||||||||
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. | |||||||||||||||||||||||||||||||||||
_tagged_fields |
|
CreateTopics Response (Version: 7) => throttle_time_ms [topics] _tagged_fields + throttle_time_ms => INT32 + topics => name topic_id error_code error_message num_partitions replication_factor [configs] _tagged_fields + name => COMPACT_STRING + topic_id => UUID + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + num_partitions => INT32 + replication_factor => INT16 + configs => name value read_only config_source is_sensitive _tagged_fields + name => COMPACT_STRING + value => COMPACT_NULLABLE_STRING + read_only => BOOLEAN + config_source => INT8 + is_sensitive => BOOLEAN +
Response header version: 1
+Field | +Description | +||||||
---|---|---|---|---|---|---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. | ||||||
topics | Results for each topic we tried to create. | ||||||
name | The topic name. | ||||||
topic_id | The unique topic ID. | ||||||
error_code | The error code, or 0 if there was no error. | ||||||
error_message | The error message, or null if there was no error. | ||||||
num_partitions | Number of partitions of the topic. | ||||||
replication_factor | Replication factor of the topic. | ||||||
configs | Configuration of the topic. | ||||||
name | The configuration name. | ||||||
value | The configuration value. | ||||||
read_only | True if the configuration is read-only. | ||||||
config_source | The configuration source. | ||||||
is_sensitive | True if this configuration is sensitive. | ||||||
_tagged_fields | The tagged fields | ||||||
_tagged_fields |
| ||||||
_tagged_fields | The tagged fields |
DeleteTopics Response (Version: 6) => throttle_time_ms [responses] _tagged_fields + throttle_time_ms => INT32 + responses => name topic_id error_code error_message _tagged_fields + name => COMPACT_NULLABLE_STRING + topic_id => UUID + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The results for each topic we tried to delete. |
name | The topic name. |
topic_id | The unique topic ID. |
error_code | The deletion error, or 0 if the deletion succeeded. |
error_message | The error message, or null if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DeleteRecords Response (Version: 2) => throttle_time_ms [topics] _tagged_fields + throttle_time_ms => INT32 + topics => name [partitions] _tagged_fields + name => COMPACT_STRING + partitions => partition_index low_watermark error_code _tagged_fields + partition_index => INT32 + low_watermark => INT64 + error_code => INT16 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Each topic that we wanted to delete records from. |
name | The topic name. |
partitions | Each partition that we wanted to delete records from. |
partition_index | The partition index. |
low_watermark | The partition low water mark. |
error_code | The deletion error code, or 0 if the deletion succeeded. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
InitProducerId Request (Version: 6) => transactional_id transaction_timeout_ms producer_id producer_epoch enable2_pc keep_prepared_txn _tagged_fields + transactional_id => COMPACT_NULLABLE_STRING + transaction_timeout_ms => INT32 + producer_id => INT64 + producer_epoch => INT16 + enable2_pc => BOOLEAN + keep_prepared_txn => BOOLEAN +
This version of the request is unstable.
Request header version: 2
+Field | +Description | +
---|---|
transactional_id | The transactional id, or null if the producer is not transactional. |
transaction_timeout_ms | The time in ms to wait before aborting idle transactions sent by this producer. This is only relevant if a TransactionalId has been defined. |
producer_id | The producer id. This is used to disambiguate requests if a transactional id is reused following its expiration. |
producer_epoch | The producer's current epoch. This will be checked against the producer epoch on the broker, and the request will return an error if they do not match. |
enable2_pc | True if the client wants to enable two-phase commit (2PC) protocol for transactions. |
keep_prepared_txn | True if the client wants to keep the currently ongoing transaction instead of aborting it. |
_tagged_fields | The tagged fields |
InitProducerId Response (Version: 0) => throttle_time_ms error_code producer_id producer_epoch throttle_time_ms => INT32 @@ -8793,6 +9984,54 @@InitProducerId API (Key: 22):
_tagged_fields The tagged fields
InitProducerId Response (Version: 5) => throttle_time_ms error_code producer_id producer_epoch _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + producer_id => INT64 + producer_epoch => INT16 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The error code, or 0 if there was no error. |
producer_id | The current producer id. |
producer_epoch | The current epoch associated with the producer id. |
_tagged_fields | The tagged fields |
InitProducerId Response (Version: 6) => throttle_time_ms error_code producer_id producer_epoch ongoing_txn_producer_id ongoing_txn_producer_epoch _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + producer_id => INT64 + producer_epoch => INT16 + ongoing_txn_producer_id => INT64 + ongoing_txn_producer_epoch => INT16 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The error code, or 0 if there was no error. |
producer_id | The current producer id. |
producer_epoch | The current epoch associated with the producer id. |
ongoing_txn_producer_id | The producer id for ongoing transaction when KeepPreparedTxn is used, -1 if there is no transaction ongoing. |
ongoing_txn_producer_epoch | The epoch associated with the producer id for ongoing transaction when KeepPreparedTxn is used, -1 if there is no transaction ongoing. |
_tagged_fields | The tagged fields |
AddPartitionsToTxn Request (Version: 0) => v3_and_below_transactional_id v3_and_below_producer_id v3_and_below_producer_epoch [v3_and_below_topics] - v3_and_below_transactional_id => STRING - v3_and_below_producer_id => INT64 - v3_and_below_producer_epoch => INT16 - v3_and_below_topics => name [partitions] ++OffsetForLeaderEpoch Response (Version: 4) => throttle_time_ms [topics] _tagged_fields + throttle_time_ms => INT32 + topics => topic [partitions] _tagged_fields + topic => COMPACT_STRING + partitions => error_code partition leader_epoch end_offset _tagged_fields + error_code => INT16 + partition => INT32 + leader_epoch => INT32 + end_offset => INT64 +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +topics Each topic we fetched offsets for. + +topic The topic name. + +partitions Each partition in the topic we fetched offsets for. + +error_code The error code 0, or if there was no error. + +partition The partition index. + +leader_epoch The leader epoch of the partition. + +end_offset The end offset of the epoch. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields AddPartitionsToTxn API (Key: 24):
+ +Requests:
++AddPartitionsToTxn Request (Version: 0) => v3_and_below_transactional_id v3_and_below_producer_id v3_and_below_producer_epoch [v3_and_below_topics] + v3_and_below_transactional_id => STRING + v3_and_below_producer_id => INT64 + v3_and_below_producer_epoch => INT16 + v3_and_below_topics => name [partitions] name => STRING partitions => INT32Request header version: 1
@@ -9276,6 +10552,48 @@AddPartitionsToTxn API (Key: 24):<
_tagged_fields The tagged fields AddPartitionsToTxn Response (Version: 5) => throttle_time_ms error_code [results_by_transaction] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + results_by_transaction => transactional_id [topic_results] _tagged_fields + transactional_id => COMPACT_STRING + topic_results => name [results_by_partition] _tagged_fields + name => COMPACT_STRING + results_by_partition => partition_index partition_error_code _tagged_fields + partition_index => INT32 + partition_error_code => INT16 +Response header version: 1
++
+Field +Description ++ +throttle_time_ms Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +error_code The response top level error code. + +results_by_transaction Results categorized by transactional ID. + +transactional_id The transactional id corresponding to the transaction. + +topic_results The results for each topic. + +name The topic name. + +results_by_partition The results for each partition. + +partition_index The partition indexes. + +partition_error_code The response error code. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields AddOffsetsToTxn API (Key: 25):
Requests:
@@ -9433,6 +10751,21 @@AddOffsetsToTxn API (Key: 25):_tagged_fields
The tagged fields
AddOffsetsToTxn Response (Version: 4) => throttle_time_ms error_code _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The response error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
EndTxn Response (Version: 5) => throttle_time_ms error_code producer_id producer_epoch _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + producer_id => INT64 + producer_epoch => INT16 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The error code, or 0 if there was no error. |
producer_id | The producer ID. |
producer_epoch | The current epoch associated with the producer. |
_tagged_fields | The tagged fields |
WriteTxnMarkers Response (Version: 1) => [markers] _tagged_fields + markers => producer_id [topics] _tagged_fields + producer_id => INT64 + topics => name [partitions] _tagged_fields + name => COMPACT_STRING + partitions => partition_index error_code _tagged_fields + partition_index => INT32 + error_code => INT16 +
Response header version: 1
+Field | +Description | +
---|---|
markers | The results for writing makers. |
producer_id | The current producer ID in use by the transactional ID. |
topics | The results by topic. |
name | The topic name. |
partitions | The results by partition. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
TxnOffsetCommit Response (Version: 5) => throttle_time_ms [topics] _tagged_fields + throttle_time_ms => INT32 + topics => name [partitions] _tagged_fields + name => COMPACT_STRING + partitions => partition_index error_code _tagged_fields + partition_index => INT32 + error_code => INT16 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses for each topic. |
name | The topic name. |
partitions | The responses for each partition in the topic. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DescribeAcls Response (Version: 3) => throttle_time_ms error_code error_message [resources] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + resources => resource_type resource_name pattern_type [acls] _tagged_fields + resource_type => INT8 + resource_name => COMPACT_STRING + pattern_type => INT8 + acls => principal host operation permission_type _tagged_fields + principal => COMPACT_STRING + host => COMPACT_STRING + operation => INT8 + permission_type => INT8 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
resources | Each Resource that is referenced in an ACL. |
resource_type | The resource type. |
resource_name | The resource name. |
pattern_type | The resource pattern type. |
acls | The ACLs. |
principal | The ACL principal. |
host | The ACL host. |
operation | The ACL operation. |
permission_type | The ACL permission type. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
CreateAcls Response (Version: 3) => throttle_time_ms [results] _tagged_fields + throttle_time_ms => INT32 + results => error_code error_message _tagged_fields + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each ACL creation. |
error_code | The result error, or zero if there was no error. |
error_message | The result message, or null if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DescribeConfigs Request (Version: 1) => [resources] include_synonyms - resources => resource_type resource_name [configuration_keys] - resource_type => INT8 - resource_name => STRING - configuration_keys => STRING ++DeleteAcls Response (Version: 3) => throttle_time_ms [filter_results] _tagged_fields + throttle_time_ms => INT32 + filter_results => error_code error_message [matching_acls] _tagged_fields + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + matching_acls => error_code error_message resource_type resource_name pattern_type principal host operation permission_type _tagged_fields + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + resource_type => INT8 + resource_name => COMPACT_STRING + pattern_type => INT8 + principal => COMPACT_STRING + host => COMPACT_STRING + operation => INT8 + permission_type => INT8 +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +filter_results The results for each filter. + +error_code The error code, or 0 if the filter succeeded. + +error_message The error message, or null if the filter succeeded. + +matching_acls The ACLs which matched this filter. + +error_code The deletion error code, or 0 if the deletion succeeded. + +error_message The deletion error message, or null if the deletion succeeded. + +resource_type The ACL resource type. + +resource_name The ACL resource name. + +pattern_type The ACL resource pattern type. + +principal The ACL principal. + +host The ACL host. + +operation The ACL operation. + +permission_type The ACL permission type. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields DescribeConfigs API (Key: 32):
+ +Requests:
++DescribeConfigs Request (Version: 1) => [resources] include_synonyms + resources => resource_type resource_name [configuration_keys] + resource_type => INT8 + resource_name => STRING + configuration_keys => STRING include_synonyms => BOOLEANRequest header version: 1
@@ -10900,6 +12448,75 @@
DescribeConfigs API (Key: 32):documentation
The configuration documentation. DescribeConfigs Response (Version: 4) => throttle_time_ms [results] _tagged_fields + throttle_time_ms => INT32 + results => error_code error_message resource_type resource_name [configs] _tagged_fields + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + resource_type => INT8 + resource_name => COMPACT_STRING + configs => name value read_only config_source is_sensitive [synonyms] config_type documentation _tagged_fields + name => COMPACT_STRING + value => COMPACT_NULLABLE_STRING + read_only => BOOLEAN + config_source => INT8 + is_sensitive => BOOLEAN + synonyms => name value source _tagged_fields + name => COMPACT_STRING + value => COMPACT_NULLABLE_STRING + source => INT8 + config_type => INT8 + documentation => COMPACT_NULLABLE_STRING +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +results The results for each resource. + +error_code The error code, or 0 if we were able to successfully describe the configurations. + +error_message The error message, or null if we were able to successfully describe the configurations. + +resource_type The resource type. + +resource_name The resource name. + +configs Each listed configuration. + +name The configuration name. + +value The configuration value. + +read_only True if the configuration is read-only. + +config_source The configuration source. + +is_sensitive True if this configuration is sensitive. + +synonyms The synonyms for this configuration key. + +name The synonym name. + +value The synonym value. + +source The synonym source. + +_tagged_fields The tagged fields + +config_type The configuration data type. Type can be one of the following values - BOOLEAN, STRING, INT, SHORT, LONG, DOUBLE, LIST, CLASS, PASSWORD. + +documentation The configuration documentation. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields AlterConfigs API (Key: 33):
Requests:
@@ -11044,6 +12661,35 @@AlterConfigs API (Key: 33):
resource_name The resource name.
AlterConfigs Response (Version: 2) => throttle_time_ms [responses] _tagged_fields + throttle_time_ms => INT32 + responses => error_code error_message resource_type resource_name _tagged_fields + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + resource_type => INT8 + resource_name => COMPACT_STRING +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The responses for each resource. |
error_code | The resource error code. |
error_message | The resource error message, or null if there was no error. |
resource_type | The resource type. |
resource_name | The resource name. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
AlterReplicaLogDirs Response (Version: 2) => throttle_time_ms [results] _tagged_fields + throttle_time_ms => INT32 + results => topic_name [partitions] _tagged_fields + topic_name => COMPACT_STRING + partitions => partition_index error_code _tagged_fields + partition_index => INT32 + error_code => INT16 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for each topic. |
topic_name | The name of the topic. |
partitions | The results for each partition. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DescribeLogDirs Response (Version: 4) => throttle_time_ms error_code [results] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + results => error_code log_dir [topics] total_bytes usable_bytes _tagged_fields + error_code => INT16 + log_dir => COMPACT_STRING + topics => name [partitions] _tagged_fields + name => COMPACT_STRING + partitions => partition_index partition_size offset_lag is_future_key _tagged_fields + partition_index => INT32 + partition_size => INT64 + offset_lag => INT64 + is_future_key => BOOLEAN + total_bytes => INT64 + usable_bytes => INT64 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The error code, or 0 if there was no error. |
results | The log directories. |
error_code | The error code, or 0 if there was no error. |
log_dir | The absolute log directory path. |
topics | The topics. |
name | The topic name. |
partitions | The partitions. |
partition_index | The partition index. |
partition_size | The size of the log segments in this partition in bytes. |
offset_lag | The lag of the log's LEO w.r.t. partition's HW (if it is the current log for the partition) or current replica's LEO (if it is the future log for the partition). |
is_future_key | True if this log is created by AlterReplicaLogDirsRequest and will replace the current log of the replica in the future. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
total_bytes | The total size in bytes of the volume the log directory is in. |
usable_bytes | The usable size in bytes of the volume the log directory is in. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
SaslAuthenticate Response (Version: 2) => error_code error_message auth_bytes session_lifetime_ms _tagged_fields + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + auth_bytes => COMPACT_BYTES + session_lifetime_ms => INT64 +
Response header version: 1
+Field | +Description | +
---|---|
error_code | The error code, or 0 if there was no error. |
error_message | The error message, or null if there was no error. |
auth_bytes | The SASL authentication bytes from the server, as defined by the SASL mechanism. |
session_lifetime_ms | Number of milliseconds after which only re-authentication over the existing connection to create a new session can occur. |
_tagged_fields | The tagged fields |
CreateDelegationToken Request (Version: 1) => [renewers] max_lifetime_ms - renewers => principal_type principal_name ++CreatePartitions Response (Version: 3) => throttle_time_ms [results] _tagged_fields + throttle_time_ms => INT32 + results => name error_code error_message _tagged_fields + name => COMPACT_STRING + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +results The partition creation results for each topic. + +name The topic name. + +error_code The result error, or zero if there was no error. + +error_message The result message, or null if there was no error. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields CreateDelegationToken API (Key: 38):
+ +Requests:
++CreateDelegationToken Request (Version: 1) => [renewers] max_lifetime_ms + renewers => principal_type principal_name principal_type => STRING principal_name => STRING max_lifetime_ms => INT64 @@ -11756,6 +13537,48 @@CreateDelegationToken API (Key:
_tagged_fields The tagged fields CreateDelegationToken Response (Version: 3) => error_code principal_type principal_name token_requester_principal_type token_requester_principal_name issue_timestamp_ms expiry_timestamp_ms max_timestamp_ms token_id hmac throttle_time_ms _tagged_fields + error_code => INT16 + principal_type => COMPACT_STRING + principal_name => COMPACT_STRING + token_requester_principal_type => COMPACT_STRING + token_requester_principal_name => COMPACT_STRING + issue_timestamp_ms => INT64 + expiry_timestamp_ms => INT64 + max_timestamp_ms => INT64 + token_id => COMPACT_STRING + hmac => COMPACT_BYTES + throttle_time_ms => INT32 +Response header version: 1
++
+Field +Description ++ +error_code The top-level error, or zero if there was no error. + +principal_type The principal type of the token owner. + +principal_name The name of the token owner. + +token_requester_principal_type The principal type of the requester of the token. + +token_requester_principal_name The principal type of the requester of the token. + +issue_timestamp_ms When this token was generated. + +expiry_timestamp_ms When this token expires. + +max_timestamp_ms The maximum lifetime of this token. + +token_id The token UUID. + +hmac HMAC of the delegation token. + +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +_tagged_fields The tagged fields RenewDelegationToken API (Key: 39):
Requests:
@@ -11804,6 +13627,24 @@RenewDelegationToken API (Key: 3
throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.
RenewDelegationToken Response (Version: 2) => error_code expiry_timestamp_ms throttle_time_ms _tagged_fields + error_code => INT16 + expiry_timestamp_ms => INT64 + throttle_time_ms => INT32 +
Response header version: 1
+Field | +Description | +
---|---|
error_code | The error code, or 0 if there was no error. |
expiry_timestamp_ms | The timestamp in milliseconds at which this token expires. |
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
_tagged_fields | The tagged fields |
ExpireDelegationToken Response (Version: 2) => error_code expiry_timestamp_ms throttle_time_ms _tagged_fields + error_code => INT16 + expiry_timestamp_ms => INT64 + throttle_time_ms => INT32 +
Response header version: 1
+Field | +Description | +
---|---|
error_code | The error code, or 0 if there was no error. |
expiry_timestamp_ms | The timestamp in milliseconds at which this token expires. |
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
_tagged_fields | The tagged fields |
DescribeDelegationToken Response (Version: 3) => error_code [tokens] throttle_time_ms _tagged_fields + error_code => INT16 + tokens => principal_type principal_name token_requester_principal_type token_requester_principal_name issue_timestamp expiry_timestamp max_timestamp token_id hmac [renewers] _tagged_fields + principal_type => COMPACT_STRING + principal_name => COMPACT_STRING + token_requester_principal_type => COMPACT_STRING + token_requester_principal_name => COMPACT_STRING + issue_timestamp => INT64 + expiry_timestamp => INT64 + max_timestamp => INT64 + token_id => COMPACT_STRING + hmac => COMPACT_BYTES + renewers => principal_type principal_name _tagged_fields + principal_type => COMPACT_STRING + principal_name => COMPACT_STRING + throttle_time_ms => INT32 +
Response header version: 1
+Field | +Description | +
---|---|
error_code | The error code, or 0 if there was no error. |
tokens | The tokens. |
principal_type | The token principal type. |
principal_name | The token principal name. |
token_requester_principal_type | The principal type of the requester of the token. |
token_requester_principal_name | The principal type of the requester of the token. |
issue_timestamp | The token issue timestamp in milliseconds. |
expiry_timestamp | The token expiry timestamp in milliseconds. |
max_timestamp | The token maximum timestamp length in milliseconds. |
token_id | The token ID. |
hmac | The token HMAC. |
renewers | Those who are able to renew this token before it expires. |
principal_type | The renewer principal type. |
principal_name | The renewer principal name. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
_tagged_fields | The tagged fields |
DeleteGroups Response (Version: 2) => throttle_time_ms [results] _tagged_fields + throttle_time_ms => INT32 + results => group_id error_code _tagged_fields + group_id => COMPACT_STRING + error_code => INT16 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The deletion results. |
group_id | The group id. |
error_code | The deletion error, or 0 if the deletion succeeded. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
ElectLeaders Response (Version: 2) => throttle_time_ms error_code [replica_election_results] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + replica_election_results => topic [partition_result] _tagged_fields + topic => COMPACT_STRING + partition_result => partition_id error_code error_message _tagged_fields + partition_id => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The top level response error code. |
replica_election_results | The election results, or an empty array if the requester did not have permission and the request asks for all partitions. |
topic | The topic name. |
partition_result | The results for each partition. |
partition_id | The partition id. |
error_code | The result error, or zero if there was no error. |
error_message | The result message, or null if there was no error. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
IncrementalAlterConfigs Response (Version: 1) => throttle_time_ms [responses] _tagged_fields + throttle_time_ms => INT32 + responses => error_code error_message resource_type resource_name _tagged_fields + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + resource_type => INT8 + resource_name => COMPACT_STRING +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
responses | The responses for each resource. |
error_code | The resource error code. |
error_message | The resource error message, or null if there was no error. |
resource_type | The resource type. |
resource_name | The resource name. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
ListPartitionReassignments Request (Version: 0) => timeout_ms [topics] _tagged_fields ++Responses:AlterPartitionReassignments Request (Version: 1) => timeout_ms allow_replication_factor_change [topics] _tagged_fields timeout_ms => INT32 - topics => name [partition_indexes] _tagged_fields + allow_replication_factor_change => BOOLEAN + topics => name [partitions] _tagged_fields name => COMPACT_STRING - partition_indexes => INT32 + partitions => partition_index [replicas] _tagged_fields + partition_index => INT32 + replicas => INT32Request header version: 2
+
Field @@ -12361,18 +14366,179 @@ListPartitionReassignments
timeout_ms The time in ms to wait for the request to complete. - +topics The topics to list partition reassignments for, or null to list everything. allow_replication_factor_change The option indicating whether changing the replication factor of any given partition as part of this request is a valid move. ++ topics The topics to reassign. name The topic name. - +partition_indexes The partitions to list partition reassignments for. partitions The partitions to reassign. ++ +partition_index The partition index. + replicas The replicas to place the partitions on, or null to cancel a pending reassignment for this partition. _tagged_fields The tagged fields - _tagged_fields The tagged fields + + +_tagged_fields The tagged fields
++AlterPartitionReassignments Response (Version: 0) => throttle_time_ms error_code error_message [responses] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + responses => name [partitions] _tagged_fields + name => COMPACT_STRING + partitions => partition_index error_code error_message _tagged_fields + partition_index => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +error_code The top-level error code, or 0 if there was no error. + +error_message The top-level error message, or null if there was no error. + +responses The responses to topics to reassign. + +name The topic name. + +partitions The responses to partitions to reassign. + +partition_index The partition index. + +error_code The error code for this partition, or 0 if there was no error. + +error_message The error message for this partition, or null if there was no error. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields +AlterPartitionReassignments Response (Version: 1) => throttle_time_ms allow_replication_factor_change error_code error_message [responses] _tagged_fields + throttle_time_ms => INT32 + allow_replication_factor_change => BOOLEAN + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + responses => name [partitions] _tagged_fields + name => COMPACT_STRING + partitions => partition_index error_code error_message _tagged_fields + partition_index => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +allow_replication_factor_change The option indicating whether changing the replication factor of any given partition as part of the request was allowed. + +error_code The top-level error code, or 0 if there was no error. + +error_message The top-level error message, or null if there was no error. + +responses The responses to topics to reassign. + +name The topic name. + +partitions The responses to partitions to reassign. + +partition_index The partition index. + +error_code The error code for this partition, or 0 if there was no error. + +error_message The error message for this partition, or null if there was no error. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields ListPartitionReassignments API (Key: 46):
+ +Requests:
+Responses:ListPartitionReassignments Request (Version: 0) => timeout_ms [topics] _tagged_fields + timeout_ms => INT32 + topics => name [partition_indexes] _tagged_fields + name => COMPACT_STRING + partition_indexes => INT32 +Request header version: 2
++
Field +Description ++ +timeout_ms The time in ms to wait for the request to complete. + +topics The topics to list partition reassignments for, or null to list everything. + +name The topic name. + +partition_indexes The partitions to list partition reassignments for. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields
+ListPartitionReassignments Response (Version: 0) => throttle_time_ms error_code error_message [topics] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + topics => name [partitions] _tagged_fields + name => COMPACT_STRING + partitions => partition_index [replicas] [adding_replicas] [removing_replicas] _tagged_fields + partition_index => INT32 + replicas => INT32 + adding_replicas => INT32 + removing_replicas => INT32 +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +error_code The top-level error code, or 0 if there was no error. + +error_message The top-level error message, or null if there was no error. + +topics The ongoing reassignments for each topic. + +name The topic name. + +partitions The ongoing reassignments for each partition. + +partition_index The index of the partition. + +replicas The current replica set. + +adding_replicas The set of replicas we are currently adding. + +removing_replicas The set of replicas we are currently removing. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields OffsetDelete API (Key: 47):
Requests:
@@ -12399,6 +14565,34 @@OffsetDelete API (Key: 47):
OffsetDelete Response (Version: 0) => error_code throttle_time_ms [topics] + error_code => INT16 + throttle_time_ms => INT32 + topics => name [partitions] + name => STRING + partitions => partition_index error_code + partition_index => INT32 + error_code => INT16 +
Response header version: 0
+Field | +Description | +
---|---|
error_code | The top-level error code, or 0 if there was no error. |
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | The responses for each topic. |
name | The topic name. |
partitions | The responses for each partition in the topic. |
partition_index | The partition index. |
error_code | The error code, or 0 if there was no error. |
DescribeClientQuotas Response (Version: 1) => throttle_time_ms error_code error_message [entries] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + entries => [entity] [values] _tagged_fields + entity => entity_type entity_name _tagged_fields + entity_type => COMPACT_STRING + entity_name => COMPACT_NULLABLE_STRING + values => key value _tagged_fields + key => COMPACT_STRING + value => FLOAT64 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The error code, or `0` if the quota description succeeded. |
error_message | The error message, or `null` if the quota description succeeded. |
entries | A result entry. |
entity | The quota entity description. |
entity_type | The entity type. |
entity_name | The entity name, or null if the default. |
_tagged_fields | The tagged fields |
values | The quota values for the entity. |
key | The quota configuration key. |
value | The quota configuration value. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
AlterClientQuotas Response (Version: 1) => throttle_time_ms [entries] _tagged_fields + throttle_time_ms => INT32 + entries => error_code error_message [entity] _tagged_fields + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + entity => entity_type entity_name _tagged_fields + entity_type => COMPACT_STRING + entity_name => COMPACT_NULLABLE_STRING +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
entries | The quota configuration entries to alter. |
error_code | The error code, or `0` if the quota alteration succeeded. |
error_message | The error message, or `null` if the quota alteration succeeded. |
entity | The quota entity to alter. |
entity_type | The entity type. |
entity_name | The name of the entity, or null if the default. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
AlterUserScramCredentials Request (Version: 0) => [deletions] [upsertions] _tagged_fields - deletions => name mechanism _tagged_fields - name => COMPACT_STRING - mechanism => INT8 - upsertions => name mechanism iterations salt salted_password _tagged_fields - name => COMPACT_STRING - mechanism => INT8 - iterations => INT32 - salt => COMPACT_BYTES - salted_password => COMPACT_BYTES -
Request header version: 2
+DescribeUserScramCredentials Response (Version: 0) => throttle_time_ms error_code error_message [results] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + results => user error_code error_message [credential_infos] _tagged_fields + user => COMPACT_STRING + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + credential_infos => mechanism iterations _tagged_fields + mechanism => INT8 + iterations => INT32 +
Response header version: 1
Field | Description |
---|---|
deletions | The SCRAM credentials to remove. | throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
name | The user name. | error_code | The message-level error code, 0 except for user authorization or infrastructure issues. |
mechanism | The SCRAM mechanism. | error_message | The message-level error message, if any. |
_tagged_fields | The tagged fields | results | The results for descriptions, one per user. | +
user | The user name. |
error_code | The user-level error code. |
error_message | The user-level error message, if any. |
credential_infos | The mechanism and related information associated with the user's SCRAM credentials. |
mechanism | The SCRAM mechanism. |
iterations | The number of iterations used in the SCRAM credential. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
AlterUserScramCredentials Request (Version: 0) => [deletions] [upsertions] _tagged_fields + deletions => name mechanism _tagged_fields + name => COMPACT_STRING + mechanism => INT8 + upsertions => name mechanism iterations salt salted_password _tagged_fields + name => COMPACT_STRING + mechanism => INT8 + iterations => INT32 + salt => COMPACT_BYTES + salted_password => COMPACT_BYTES +
Request header version: 2
+Field | +Description | +
---|---|
deletions | The SCRAM credentials to remove. |
name | The user name. |
mechanism | The SCRAM mechanism. |
_tagged_fields | The tagged fields |
upsertions | The SCRAM credentials to update/insert. |
AlterUserScramCredentials Response (Version: 0) => throttle_time_ms [results] _tagged_fields + throttle_time_ms => INT32 + results => user error_code error_message _tagged_fields + user => COMPACT_STRING + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
results | The results for deletions and alterations, one per affected user. |
user | The user name. |
error_code | The error code. |
error_message | The error message, if any. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DescribeQuorum Response (Version: 2) => error_code error_message [topics] [nodes] _tagged_fields + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + topics => topic_name [partitions] _tagged_fields + topic_name => COMPACT_STRING + partitions => partition_index error_code error_message leader_id leader_epoch high_watermark [current_voters] [observers] _tagged_fields + partition_index => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + leader_id => INT32 + leader_epoch => INT32 + high_watermark => INT64 + current_voters => replica_id replica_directory_id log_end_offset last_fetch_timestamp last_caught_up_timestamp _tagged_fields + replica_id => INT32 + replica_directory_id => UUID + log_end_offset => INT64 + last_fetch_timestamp => INT64 + last_caught_up_timestamp => INT64 + observers => replica_id replica_directory_id log_end_offset last_fetch_timestamp last_caught_up_timestamp _tagged_fields + replica_id => INT32 + replica_directory_id => UUID + log_end_offset => INT64 + last_fetch_timestamp => INT64 + last_caught_up_timestamp => INT64 + nodes => node_id [listeners] _tagged_fields + node_id => INT32 + listeners => name host port _tagged_fields + name => COMPACT_STRING + host => COMPACT_STRING + port => UINT16 +
Response header version: 1
+Field | +Description | +
---|---|
error_code | The top level error code. |
error_message | The error message, or null if there was no error. |
topics | The response from the describe quorum API. |
topic_name | The topic name. |
partitions | The partition data. |
partition_index | The partition index. |
error_code | The partition error code. |
error_message | The error message, or null if there was no error. |
leader_id | The ID of the current leader or -1 if the leader is unknown. |
leader_epoch | The latest known leader epoch. |
high_watermark | The high water mark. |
current_voters | The current voters of the partition. |
replica_id | The ID of the replica. |
replica_directory_id | The replica directory ID of the replica. |
log_end_offset | The last known log end offset of the follower or -1 if it is unknown. |
last_fetch_timestamp | The last known leader wall clock time time when a follower fetched from the leader. This is reported as -1 both for the current leader or if it is unknown for a voter. |
last_caught_up_timestamp | The leader wall clock append time of the offset for which the follower made the most recent fetch request. This is reported as the current time for the leader and -1 if unknown for a voter. |
_tagged_fields | The tagged fields |
observers | The observers of the partition. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
nodes | The nodes in the quorum. |
node_id | The ID of the associated node. |
listeners | The listeners of this controller. |
name | The name of the endpoint. |
host | The hostname. |
port | The port. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
UpdateFeatures Response (Version: 2) => throttle_time_ms error_code error_message _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The top-level error code, or `0` if there was no top-level error. |
error_message | The top-level error message, or `null` if there was no top-level error. |
_tagged_fields | The tagged fields |
DescribeCluster Response (Version: 2) => throttle_time_ms error_code error_message endpoint_type cluster_id controller_id [brokers] cluster_authorized_operations _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + endpoint_type => INT8 + cluster_id => COMPACT_STRING + controller_id => INT32 + brokers => broker_id host port rack is_fenced _tagged_fields + broker_id => INT32 + host => COMPACT_STRING + port => INT32 + rack => COMPACT_NULLABLE_STRING + is_fenced => BOOLEAN + cluster_authorized_operations => INT32 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The top-level error code, or 0 if there was no error. |
error_message | The top-level error message, or null if there was no error. |
endpoint_type | The endpoint type that was described. 1=brokers, 2=controllers. |
cluster_id | The cluster ID that responding broker belongs to. |
controller_id | The ID of the controller broker. |
brokers | Each broker in the response. |
broker_id | The broker ID. |
host | The broker hostname. |
port | The broker port. |
rack | The rack of the broker, or null if it has not been assigned to a rack. |
is_fenced | Whether the broker is fenced |
_tagged_fields | The tagged fields |
cluster_authorized_operations | 32-bit bitfield to represent authorized operations for this cluster. |
_tagged_fields | The tagged fields |
UnregisterBroker Request (Version: 0) => broker_id _tagged_fields - broker_id => INT32 -
Request header version: 2
+DescribeProducers Response (Version: 0) => throttle_time_ms [topics] _tagged_fields + throttle_time_ms => INT32 + topics => name [partitions] _tagged_fields + name => COMPACT_STRING + partitions => partition_index error_code error_message [active_producers] _tagged_fields + partition_index => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + active_producers => producer_id producer_epoch last_sequence last_timestamp coordinator_epoch current_txn_start_offset _tagged_fields + producer_id => INT64 + producer_epoch => INT32 + last_sequence => INT32 + last_timestamp => INT64 + coordinator_epoch => INT32 + current_txn_start_offset => INT64 +
Response header version: 1
Field | Description |
---|---|
broker_id | The broker ID to unregister. | throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. | +
topics | Each topic in the response. |
name | The topic name. |
partitions | Each partition in the response. |
partition_index | The partition index. |
error_code | The partition error code, or 0 if there was no error. |
error_message | The partition error message, which may be null if no additional details are available. |
active_producers | The active producers for the partition. |
producer_id | The producer id. |
producer_epoch | The producer epoch. |
last_sequence | The last sequence number sent by the producer. |
last_timestamp | The last timestamp sent by the producer. |
coordinator_epoch | The current epoch of the producer group. |
current_txn_start_offset | The current transaction start offset of the producer. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
DescribeTransactions Request (Version: 0) => [transactional_ids] _tagged_fields - transactional_ids => COMPACT_STRING +Responses:UnregisterBroker Request (Version: 0) => broker_id _tagged_fields + broker_id => INT32Request header version: 2
Field Description - +transactional_ids Array of transactionalIds to include in describe results. If empty, then no results will be returned. broker_id The broker ID to unregister. _tagged_fields The tagged fields
++UnregisterBroker Response (Version: 0) => throttle_time_ms error_code error_message _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +Response header version: 1
++
+Field +Description ++ +throttle_time_ms Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +error_code The error code, or 0 if there was no error. + +error_message The top-level error message, or `null` if there was no top-level error. + +_tagged_fields The tagged fields DescribeTransactions API (Key: 65):
+ +Requests:
++Responses:DescribeTransactions Request (Version: 0) => [transactional_ids] _tagged_fields + transactional_ids => COMPACT_STRING +Request header version: 2
++
+Field +Description ++ +transactional_ids Array of transactionalIds to include in describe results. If empty, then no results will be returned. + +_tagged_fields The tagged fields
+DescribeTransactions Response (Version: 0) => throttle_time_ms [transaction_states] _tagged_fields + throttle_time_ms => INT32 + transaction_states => error_code transactional_id transaction_state transaction_timeout_ms transaction_start_time_ms producer_id producer_epoch [topics] _tagged_fields + error_code => INT16 + transactional_id => COMPACT_STRING + transaction_state => COMPACT_STRING + transaction_timeout_ms => INT32 + transaction_start_time_ms => INT64 + producer_id => INT64 + producer_epoch => INT16 + topics => topic [partitions] _tagged_fields + topic => COMPACT_STRING + partitions => INT32 +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +transaction_states The current state of the transaction. + +error_code The error code. + +transactional_id The transactional id. + +transaction_state The current transaction state of the producer. + +transaction_timeout_ms The timeout in milliseconds for the transaction. + +transaction_start_time_ms The start time of the transaction in milliseconds. + +producer_id The current producer id associated with the transaction. + +producer_epoch The current epoch associated with the producer id. + +topics The set of partitions included in the current transaction (if active). When a transaction is preparing to commit or abort, this will include only partitions which do not have markers. + +topic The topic name. + +partitions The partition ids included in the current transaction. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields ListTransactions API (Key: 66):
Requests:
@@ -13244,6 +15874,27 @@ListTransactions API (Key: 66):<
_tagged_fields The tagged fields
ListTransactions Request (Version: 2) => [state_filters] [producer_id_filters] duration_filter transactional_id_pattern _tagged_fields + state_filters => COMPACT_STRING + producer_id_filters => INT64 + duration_filter => INT64 + transactional_id_pattern => COMPACT_NULLABLE_STRING +
Request header version: 2
+Field | +Description | +
---|---|
state_filters | The transaction states to filter by: if empty, all transactions are returned; if non-empty, then only transactions matching one of the filtered states will be returned. |
producer_id_filters | The producerIds to filter by: if empty, all transactions will be returned; if non-empty, only transactions which match one of the filtered producerIds will be returned. |
duration_filter | Duration (in millis) to filter by: if < 0, all transactions will be returned; otherwise, only transactions running longer than this duration will be returned. |
transactional_id_pattern | The transactional ID regular expression pattern to filter by: if it is empty or null, all transactions are returned; Otherwise then only the transactions matching the given regular expression will be returned. |
_tagged_fields | The tagged fields |
ListTransactions Response (Version: 0) => throttle_time_ms error_code [unknown_state_filters] [transaction_states] _tagged_fields throttle_time_ms => INT32 @@ -13277,6 +15928,70 @@ListTransactions API (Key: 66):<
_tagged_fields The tagged fields
ListTransactions Response (Version: 1) => throttle_time_ms error_code [unknown_state_filters] [transaction_states] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + unknown_state_filters => COMPACT_STRING + transaction_states => transactional_id producer_id transaction_state _tagged_fields + transactional_id => COMPACT_STRING + producer_id => INT64 + transaction_state => COMPACT_STRING +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The error code, or 0 if there was no error. |
unknown_state_filters | Set of state filters provided in the request which were unknown to the transaction coordinator. |
transaction_states | The current state of the transaction for the transactional id. |
transactional_id | The transactional id. |
producer_id | The producer id. |
transaction_state | The current transaction state of the producer. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
ListTransactions Response (Version: 2) => throttle_time_ms error_code [unknown_state_filters] [transaction_states] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + unknown_state_filters => COMPACT_STRING + transaction_states => transactional_id producer_id transaction_state _tagged_fields + transactional_id => COMPACT_STRING + producer_id => INT64 + transaction_state => COMPACT_STRING +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The error code, or 0 if there was no error. |
unknown_state_filters | Set of state filters provided in the request which were unknown to the transaction coordinator. |
transaction_states | The current state of the transaction for the transactional id. |
transactional_id | The transactional id. |
producer_id | The producer id. |
transaction_state | The current transaction state of the producer. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
ConsumerGroupHeartbeat Response (Version: 1) => throttle_time_ms error_code error_message member_id member_epoch heartbeat_interval_ms assignment _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + member_id => COMPACT_NULLABLE_STRING + member_epoch => INT32 + heartbeat_interval_ms => INT32 + assignment => [topic_partitions] _tagged_fields + topic_partitions => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => INT32 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The top-level error code, or 0 if there was no error. |
error_message | The top-level error message, or null if there was no error. |
member_id | The member id is generated by the consumer starting from version 1, while in version 0, it can be provided by users or generated by the group coordinator. |
member_epoch | The member epoch. |
heartbeat_interval_ms | The heartbeat interval in milliseconds. |
assignment | null if not provided; the assignment otherwise. |
topic_partitions | The partitions assigned to the member that can be used immediately. |
topic_id | The topic ID. |
partitions | The partitions. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
GetTelemetrySubscriptions Request (Version: 0) => client_instance_id _tagged_fields - client_instance_id => UUID -
Request header version: 2
+ConsumerGroupDescribe Response (Version: 1) => throttle_time_ms [groups] _tagged_fields + throttle_time_ms => INT32 + groups => error_code error_message group_id group_state group_epoch assignment_epoch assignor_name [members] authorized_operations _tagged_fields + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + group_id => COMPACT_STRING + group_state => COMPACT_STRING + group_epoch => INT32 + assignment_epoch => INT32 + assignor_name => COMPACT_STRING + members => member_id instance_id rack_id member_epoch client_id client_host [subscribed_topic_names] subscribed_topic_regex assignment target_assignment member_type _tagged_fields + member_id => COMPACT_STRING + instance_id => COMPACT_NULLABLE_STRING + rack_id => COMPACT_NULLABLE_STRING + member_epoch => INT32 + client_id => COMPACT_STRING + client_host => COMPACT_STRING + subscribed_topic_names => COMPACT_STRING + subscribed_topic_regex => COMPACT_NULLABLE_STRING + assignment => [topic_partitions] _tagged_fields + topic_partitions => topic_id topic_name [partitions] _tagged_fields + topic_id => UUID + topic_name => COMPACT_STRING + partitions => INT32 + target_assignment => [topic_partitions] _tagged_fields + topic_partitions => topic_id topic_name [partitions] _tagged_fields + topic_id => UUID + topic_name => COMPACT_STRING + partitions => INT32 + member_type => INT8 + authorized_operations => INT32 +
Response header version: 1
Field | Description |
---|---|
client_instance_id | Unique id for this client instance, must be set to 0 on the first request. | throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
_tagged_fields | The tagged fields |
PushTelemetry Request (Version: 0) => client_instance_id subscription_id terminating compression_type metrics _tagged_fields - client_instance_id => UUID - subscription_id => INT32 - terminating => BOOLEAN - compression_type => INT8 - metrics => COMPACT_BYTES -
Request header version: 2
-Field | -Description | -
---|---|
client_instance_id | Unique id for this client instance. | groups | Each described group. |
subscription_id | Unique identifier for the current subscription. | error_code | The describe error, or 0 if there was no error. |
terminating | Client is terminating the connection. | error_message | The top-level error message, or null if there was no error. |
compression_type | Compression codec used to compress the metrics. | group_id | The group ID string. |
metrics | Metrics encoded in OpenTelemetry MetricsData v1 protobuf format. | group_state | The group state string, or the empty string. | +
group_epoch | The group epoch. |
assignment_epoch | The assignment epoch. |
assignor_name | The selected assignor. |
members | The members. |
member_id | The member ID. |
instance_id | The member instance ID. |
rack_id | The member rack ID. |
member_epoch | The current member epoch. |
client_id | The client ID. |
client_host | The client host. |
subscribed_topic_names | The subscribed topic names. |
subscribed_topic_regex | the subscribed topic regex otherwise or null of not provided. |
assignment | The current assignment. |
topic_partitions | The assigned topic-partitions to the member. |
topic_id | The topic ID. |
topic_name | The topic name. |
partitions | The partitions. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
target_assignment | The target assignment. |
member_type | -1 for unknown. 0 for classic member. +1 for consumer member. |
_tagged_fields | The tagged fields |
authorized_operations | 32-bit bitfield to represent authorized operations for this group. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
GetTelemetrySubscriptions Request (Version: 0) => client_instance_id _tagged_fields + client_instance_id => UUID +
Request header version: 2
+Field | +Description | +
---|---|
client_instance_id | Unique id for this client instance, must be set to 0 on the first request. |
_tagged_fields | The tagged fields |
GetTelemetrySubscriptions Response (Version: 0) => throttle_time_ms error_code client_instance_id subscription_id [accepted_compression_types] push_interval_ms telemetry_max_bytes delta_temporality [requested_metrics] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + client_instance_id => UUID + subscription_id => INT32 + accepted_compression_types => INT8 + push_interval_ms => INT32 + telemetry_max_bytes => INT32 + delta_temporality => BOOLEAN + requested_metrics => COMPACT_STRING +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The error code, or 0 if there was no error. |
client_instance_id | Assigned client instance id if ClientInstanceId was 0 in the request, else 0. |
subscription_id | Unique identifier for the current subscription set for this client instance. |
accepted_compression_types | Compression types that broker accepts for the PushTelemetryRequest. |
push_interval_ms | Configured push interval, which is the lowest configured interval in the current subscription set. |
telemetry_max_bytes | The maximum bytes of binary data the broker accepts in PushTelemetryRequest. |
delta_temporality | Flag to indicate monotonic/counter metrics are to be emitted as deltas or cumulative values. |
requested_metrics | Requested metrics prefix string match. Empty array: No metrics subscribed, Array[0] empty string: All metrics subscribed. |
_tagged_fields | The tagged fields |
PushTelemetry Request (Version: 0) => client_instance_id subscription_id terminating compression_type metrics _tagged_fields + client_instance_id => UUID + subscription_id => INT32 + terminating => BOOLEAN + compression_type => INT8 + metrics => COMPACT_BYTES +
Request header version: 2
+Field | +Description | +
---|---|
client_instance_id | Unique id for this client instance. |
subscription_id | Unique identifier for the current subscription. |
terminating | Client is terminating the connection. |
compression_type | Compression codec used to compress the metrics. |
metrics | Metrics encoded in OpenTelemetry MetricsData v1 protobuf format. |
_tagged_fields | The tagged fields |
PushTelemetry Response (Version: 0) => throttle_time_ms error_code _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
error_code | The error code, or 0 if there was no error. |
_tagged_fields | The tagged fields |
ListClientMetricsResources Request (Version: 0) => _tagged_fields ++ListConfigResources Request (Version: 0) => _tagged_fields +Request header version: 2
++
+Field +Description ++ +_tagged_fields The tagged fields Responses:ListConfigResources Request (Version: 1) => [resource_types] _tagged_fields + resource_types => INT8Request header version: 2
Field Description + +resource_types The list of resource type. If the list is empty, it uses default supported config resource types. _tagged_fields The tagged fields
++ListConfigResources Response (Version: 0) => throttle_time_ms error_code [config_resources] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + config_resources => resource_name _tagged_fields + resource_name => COMPACT_STRING +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +error_code The error code, or 0 if there was no error. + +config_resources Each config resource in the response. + +resource_name The resource name. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields ListConfigResources Response (Version: 1) => throttle_time_ms error_code [config_resources] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + config_resources => resource_name resource_type _tagged_fields + resource_name => COMPACT_STRING + resource_type => INT8 +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +error_code The error code, or 0 if there was no error. + +config_resources Each config resource in the response. + +resource_name The resource name. + +resource_type The resource type. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields DescribeTopicPartitions API (Key: 75):
Requests:
@@ -13637,23 +16606,98 @@DescribeTopicPartitions API (
DescribeTopicPartitions Response (Version: 0) => throttle_time_ms [topics] next_cursor _tagged_fields + throttle_time_ms => INT32 + topics => error_code name topic_id is_internal [partitions] topic_authorized_operations _tagged_fields + error_code => INT16 + name => COMPACT_NULLABLE_STRING + topic_id => UUID + is_internal => BOOLEAN + partitions => error_code partition_index leader_id leader_epoch [replica_nodes] [isr_nodes] [eligible_leader_replicas] [last_known_elr] [offline_replicas] _tagged_fields + error_code => INT16 + partition_index => INT32 + leader_id => INT32 + leader_epoch => INT32 + replica_nodes => INT32 + isr_nodes => INT32 + eligible_leader_replicas => INT32 + last_known_elr => INT32 + offline_replicas => INT32 + topic_authorized_operations => INT32 + next_cursor => topic_name partition_index _tagged_fields + topic_name => COMPACT_STRING + partition_index => INT32 +
Response header version: 1
+Field | +Description | +
---|---|
throttle_time_ms | The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. |
topics | Each topic in the response. |
error_code | The topic error, or 0 if there was no error. |
name | The topic name. |
topic_id | The topic id. |
is_internal | True if the topic is internal. |
partitions | Each partition in the topic. |
error_code | The partition error, or 0 if there was no error. |
partition_index | The partition index. |
leader_id | The ID of the leader broker. |
leader_epoch | The leader epoch of this partition. |
replica_nodes | The set of all nodes that host this partition. |
isr_nodes | The set of nodes that are in sync with the leader for this partition. |
eligible_leader_replicas | The new eligible leader replicas otherwise. |
last_known_elr | The last known ELR. |
offline_replicas | The set of offline replicas of this partition. |
_tagged_fields | The tagged fields |
topic_authorized_operations | 32-bit bitfield to represent authorized operations for this topic. |
_tagged_fields | The tagged fields |
next_cursor | The next topic and partition index to fetch details for. |
topic_name | The name for the first topic to process. |
partition_index | The partition index to start with. |
_tagged_fields | The tagged fields |
_tagged_fields | The tagged fields |
ShareGroupHeartbeat Request (Version: 0) => group_id member_id member_epoch rack_id [subscribed_topic_names] _tagged_fields +Responses:ShareGroupHeartbeat Request (Version: 1) => group_id member_id member_epoch rack_id [subscribed_topic_names] _tagged_fields group_id => COMPACT_STRING member_id => COMPACT_STRING member_epoch => INT32 rack_id => COMPACT_NULLABLE_STRING subscribed_topic_names => COMPACT_STRING -This version of the request is unstable.
Request header version: 2
+Request header version: 2
Field Description group_id The group identifier. - +member_id The member id. member_id The member id generated by the consumer. The member id must be kept during the entire lifetime of the consumer process. member_epoch The current member epoch; 0 to join the group; -1 to leave the group. @@ -13665,13 +16709,56 @@ ShareGroupHeartbeat API (Key: 76)
-ShareGroupDescribe API (Key: 77):
- -Requests:
-ShareGroupDescribe Request (Version: 0) => [group_ids] include_authorized_operations _tagged_fields - group_ids => COMPACT_STRING - include_authorized_operations => BOOLEAN -This version of the request is unstable.
Request header version: 2
++ShareGroupHeartbeat Response (Version: 1) => throttle_time_ms error_code error_message member_id member_epoch heartbeat_interval_ms assignment _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + member_id => COMPACT_NULLABLE_STRING + member_epoch => INT32 + heartbeat_interval_ms => INT32 + assignment => [topic_partitions] _tagged_fields + topic_partitions => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => INT32 +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +error_code The top-level error code, or 0 if there was no error. + +error_message The top-level error message, or null if there was no error. + +member_id The member ID is generated by the consumer and provided by the consumer for all requests. + +member_epoch The member epoch. + +heartbeat_interval_ms The heartbeat interval in milliseconds. + +assignment null if not provided; the assignment otherwise. + +topic_partitions The partitions assigned to the member. + +topic_id The topic ID. + +partitions The partitions. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields ShareGroupDescribe API (Key: 77):
+ +Requests:
+Responses:ShareGroupDescribe Request (Version: 1) => [group_ids] include_authorized_operations _tagged_fields + group_ids => COMPACT_STRING + include_authorized_operations => BOOLEAN +Request header version: 2
Field Description @@ -13684,21 +16771,105 @@ShareGroupDescribe API (Key: 77):<
+ShareGroupDescribe Response (Version: 1) => throttle_time_ms [groups] _tagged_fields + throttle_time_ms => INT32 + groups => error_code error_message group_id group_state group_epoch assignment_epoch assignor_name [members] authorized_operations _tagged_fields + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + group_id => COMPACT_STRING + group_state => COMPACT_STRING + group_epoch => INT32 + assignment_epoch => INT32 + assignor_name => COMPACT_STRING + members => member_id rack_id member_epoch client_id client_host [subscribed_topic_names] assignment _tagged_fields + member_id => COMPACT_STRING + rack_id => COMPACT_NULLABLE_STRING + member_epoch => INT32 + client_id => COMPACT_STRING + client_host => COMPACT_STRING + subscribed_topic_names => COMPACT_STRING + assignment => [topic_partitions] _tagged_fields + topic_partitions => topic_id topic_name [partitions] _tagged_fields + topic_id => UUID + topic_name => COMPACT_STRING + partitions => INT32 + authorized_operations => INT32 +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +groups Each described group. + +error_code The describe error, or 0 if there was no error. + +error_message The top-level error message, or null if there was no error. + +group_id The group ID string. + +group_state The group state string, or the empty string. + +group_epoch The group epoch. + +assignment_epoch The assignment epoch. + +assignor_name The selected assignor. + +members The members. + +member_id The member ID. + +rack_id The member rack ID. + +member_epoch The current member epoch. + +client_id The client ID. + +client_host The client host. + +subscribed_topic_names The subscribed topic names. + +assignment The current assignment. + +topic_partitions The assigned topic-partitions to the member. + +topic_id The topic ID. + +topic_name The topic name. + +partitions The partitions. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +authorized_operations 32-bit bitfield to represent authorized operations for this group. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields ShareFetch API (Key: 78):
Requests:
-ShareFetch Request (Version: 0) => group_id member_id share_session_epoch max_wait_ms min_bytes max_bytes [topics] [forgotten_topics_data] _tagged_fields ++Responses:ShareFetch Request (Version: 1) => group_id member_id share_session_epoch max_wait_ms min_bytes max_bytes max_records batch_size [topics] [forgotten_topics_data] _tagged_fields group_id => COMPACT_NULLABLE_STRING member_id => COMPACT_NULLABLE_STRING share_session_epoch => INT32 max_wait_ms => INT32 min_bytes => INT32 max_bytes => INT32 + max_records => INT32 + batch_size => INT32 topics => topic_id [partitions] _tagged_fields topic_id => UUID - partitions => partition_index partition_max_bytes [acknowledgement_batches] _tagged_fields + partitions => partition_index [acknowledgement_batches] _tagged_fields partition_index => INT32 - partition_max_bytes => INT32 acknowledgement_batches => first_offset last_offset [acknowledge_types] _tagged_fields first_offset => INT64 last_offset => INT64 @@ -13706,7 +16877,7 @@ShareFetch API (Key: 78):
forgotten_topics_data => topic_id [partitions] _tagged_fields topic_id => UUID partitions => INT32 -This version of the request is unstable.
Request header version: 2
+Request header version: 2
+
Field Description @@ -13721,7 +16892,11 @@ShareFetch API (Key: 78):
min_bytes The minimum bytes to accumulate in the response. - +max_bytes The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored. max_bytes The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored. ++ +max_records The maximum number of records to fetch. This limit can be exceeded for alignment of batch boundaries. + batch_size The optimal number of records for batches of acquired records and acknowledgements. topics The topics to fetch. @@ -13731,118 +16906,1242 @@ ShareFetch API (Key: 78):
partition_index The partition index. - +partition_max_bytes The maximum bytes to fetch from this partition. 0 when only acknowledgement with no fetching is required. See KIP-74 for cases where this limit may not be honored. acknowledgement_batches Record batches to acknowledge. ++ +first_offset First offset of batch of records to acknowledge. + +last_offset Last offset (inclusive) of batch of records to acknowledge. + +acknowledge_types Array of acknowledge types - 0:Gap,1:Accept,2:Release,3:Reject. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +forgotten_topics_data The partitions to remove from this share session. + +topic_id The unique topic ID. + +partitions The partitions indexes to forget. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields
++ShareFetch Response (Version: 1) => throttle_time_ms error_code error_message acquisition_lock_timeout_ms [responses] [node_endpoints] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + acquisition_lock_timeout_ms => INT32 + responses => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition_index error_code error_message acknowledge_error_code acknowledge_error_message current_leader records [acquired_records] _tagged_fields + partition_index => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + acknowledge_error_code => INT16 + acknowledge_error_message => COMPACT_NULLABLE_STRING + current_leader => leader_id leader_epoch _tagged_fields + leader_id => INT32 + leader_epoch => INT32 + records => COMPACT_RECORDS + acquired_records => first_offset last_offset delivery_count _tagged_fields + first_offset => INT64 + last_offset => INT64 + delivery_count => INT16 + node_endpoints => node_id host port rack _tagged_fields + node_id => INT32 + host => COMPACT_STRING + port => INT32 + rack => COMPACT_NULLABLE_STRING +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +error_code The top-level response error code. + +error_message The top-level error message, or null if there was no error. + +acquisition_lock_timeout_ms The time in milliseconds for which the acquired records are locked. + +responses The response topics. + +topic_id The unique topic ID. + +partitions The topic partitions. + +partition_index The partition index. + +error_code The fetch error code, or 0 if there was no fetch error. + +error_message The fetch error message, or null if there was no fetch error. + +acknowledge_error_code The acknowledge error code, or 0 if there was no acknowledge error. + +acknowledge_error_message The acknowledge error message, or null if there was no acknowledge error. + +current_leader The current leader of the partition. + +leader_id The ID of the current leader or -1 if the leader is unknown. + +leader_epoch The latest known leader epoch. + +_tagged_fields The tagged fields + +records The record data. + +acquired_records The acquired records. + +first_offset The earliest offset in this batch of acquired records. + +last_offset The last offset of this batch of acquired records. + +delivery_count The delivery count of this batch of acquired records. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +node_endpoints Endpoints for all current leaders enumerated in PartitionData with error NOT_LEADER_OR_FOLLOWER. + +node_id The ID of the associated node. + +host The node's hostname. + +port The node's port. + +rack The rack of the node, or null if it has not been assigned to a rack. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields ShareAcknowledge API (Key: 79):
+ +Requests:
++Responses:ShareAcknowledge Request (Version: 1) => group_id member_id share_session_epoch [topics] _tagged_fields + group_id => COMPACT_NULLABLE_STRING + member_id => COMPACT_NULLABLE_STRING + share_session_epoch => INT32 + topics => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition_index [acknowledgement_batches] _tagged_fields + partition_index => INT32 + acknowledgement_batches => first_offset last_offset [acknowledge_types] _tagged_fields + first_offset => INT64 + last_offset => INT64 + acknowledge_types => INT8 +Request header version: 2
++
+Field +Description ++ +group_id The group identifier. + +member_id The member ID. + +share_session_epoch The current share session epoch: 0 to open a share session; -1 to close it; otherwise increments for consecutive requests. + +topics The topics containing records to acknowledge. + +topic_id The unique topic ID. + +partitions The partitions containing records to acknowledge. + +partition_index The partition index. + +acknowledgement_batches Record batches to acknowledge. + +first_offset First offset of batch of records to acknowledge. + +last_offset Last offset (inclusive) of batch of records to acknowledge. + +acknowledge_types Array of acknowledge types - 0:Gap,1:Accept,2:Release,3:Reject. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields
++ShareAcknowledge Response (Version: 1) => throttle_time_ms error_code error_message [responses] [node_endpoints] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + responses => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition_index error_code error_message current_leader _tagged_fields + partition_index => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + current_leader => leader_id leader_epoch _tagged_fields + leader_id => INT32 + leader_epoch => INT32 + node_endpoints => node_id host port rack _tagged_fields + node_id => INT32 + host => COMPACT_STRING + port => INT32 + rack => COMPACT_NULLABLE_STRING +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +error_code The top level response error code. + +error_message The top-level error message, or null if there was no error. + +responses The response topics. + +topic_id The unique topic ID. + +partitions The topic partitions. + +partition_index The partition index. + +error_code The error code, or 0 if there was no error. + +error_message The error message, or null if there was no error. + +current_leader The current leader of the partition. + +leader_id The ID of the current leader or -1 if the leader is unknown. + +leader_epoch The latest known leader epoch. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +node_endpoints Endpoints for all current leaders enumerated in PartitionData with error NOT_LEADER_OR_FOLLOWER. + +node_id The ID of the associated node. + +host The node's hostname. + +port The node's port. + +rack The rack of the node, or null if it has not been assigned to a rack. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields AddRaftVoter API (Key: 80):
+ +Requests:
++Responses:AddRaftVoter Request (Version: 0) => cluster_id timeout_ms voter_id voter_directory_id [listeners] _tagged_fields + cluster_id => COMPACT_NULLABLE_STRING + timeout_ms => INT32 + voter_id => INT32 + voter_directory_id => UUID + listeners => name host port _tagged_fields + name => COMPACT_STRING + host => COMPACT_STRING + port => UINT16 +Request header version: 2
++
+Field +Description ++ +cluster_id The cluster id. + +timeout_ms The maximum time to wait for the request to complete before returning. + +voter_id The replica id of the voter getting added to the topic partition. + +voter_directory_id The directory id of the voter getting added to the topic partition. + +listeners The endpoints that can be used to communicate with the voter. + +name The name of the endpoint. + +host The hostname. + +port The port. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields
++AddRaftVoter Response (Version: 0) => throttle_time_ms error_code error_message _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +error_code The error code, or 0 if there was no error. + +error_message The error message, or null if there was no error. + +_tagged_fields The tagged fields RemoveRaftVoter API (Key: 81):
+ +Requests:
++Responses:RemoveRaftVoter Request (Version: 0) => cluster_id voter_id voter_directory_id _tagged_fields + cluster_id => COMPACT_NULLABLE_STRING + voter_id => INT32 + voter_directory_id => UUID +Request header version: 2
++
+Field +Description ++ +cluster_id The cluster id of the request. + +voter_id The replica id of the voter getting removed from the topic partition. + +voter_directory_id The directory id of the voter getting removed from the topic partition. + +_tagged_fields The tagged fields
++RemoveRaftVoter Response (Version: 0) => throttle_time_ms error_code error_message _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +error_code The error code, or 0 if there was no error. + +error_message The error message, or null if there was no error. + +_tagged_fields The tagged fields InitializeShareGroupState API (Key: 83):
+ +Requests:
++Responses:InitializeShareGroupState Request (Version: 0) => group_id [topics] _tagged_fields + group_id => COMPACT_STRING + topics => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition state_epoch start_offset _tagged_fields + partition => INT32 + state_epoch => INT32 + start_offset => INT64 +Request header version: 2
++
+Field +Description ++ +group_id The group identifier. + +topics The data for the topics. + +topic_id The topic identifier. + +partitions The data for the partitions. + +partition The partition index. + +state_epoch The state epoch for this share-partition. + +start_offset The share-partition start offset, or -1 if the start offset is not being initialized. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields
++InitializeShareGroupState Response (Version: 0) => [results] _tagged_fields + results => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition error_code error_message _tagged_fields + partition => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +Response header version: 1
++
+Field +Description ++ +results The initialization results. + +topic_id The topic identifier. + +partitions The results for the partitions. + +partition The partition index. + +error_code The error code, or 0 if there was no error. + +error_message The error message, or null if there was no error. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields ReadShareGroupState API (Key: 84):
+ +Requests:
++Responses:ReadShareGroupState Request (Version: 0) => group_id [topics] _tagged_fields + group_id => COMPACT_STRING + topics => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition leader_epoch _tagged_fields + partition => INT32 + leader_epoch => INT32 +Request header version: 2
++
+Field +Description ++ +group_id The group identifier. + +topics The data for the topics. + +topic_id The topic identifier. + +partitions The data for the partitions. + +partition The partition index. + +leader_epoch The leader epoch of the share-partition. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields
++ReadShareGroupState Response (Version: 0) => [results] _tagged_fields + results => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition error_code error_message state_epoch start_offset [state_batches] _tagged_fields + partition => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + state_epoch => INT32 + start_offset => INT64 + state_batches => first_offset last_offset delivery_state delivery_count _tagged_fields + first_offset => INT64 + last_offset => INT64 + delivery_state => INT8 + delivery_count => INT16 +Response header version: 1
++
+Field +Description ++ +results The read results. + +topic_id The topic identifier. + +partitions The results for the partitions. + +partition The partition index. + +error_code The error code, or 0 if there was no error. + +error_message The error message, or null if there was no error. + +state_epoch The state epoch of the share-partition. + +start_offset The share-partition start offset, which can be -1 if it is not yet initialized. + +state_batches The state batches for this share-partition. + +first_offset The first offset of this state batch. + +last_offset The last offset of this state batch. + +delivery_state The delivery state - 0:Available,2:Acked,4:Archived. + +delivery_count The delivery count. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields WriteShareGroupState API (Key: 85):
+ +Requests:
++Responses:WriteShareGroupState Request (Version: 0) => group_id [topics] _tagged_fields + group_id => COMPACT_STRING + topics => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition state_epoch leader_epoch start_offset [state_batches] _tagged_fields + partition => INT32 + state_epoch => INT32 + leader_epoch => INT32 + start_offset => INT64 + state_batches => first_offset last_offset delivery_state delivery_count _tagged_fields + first_offset => INT64 + last_offset => INT64 + delivery_state => INT8 + delivery_count => INT16 +Request header version: 2
++
+Field +Description ++ +group_id The group identifier. + +topics The data for the topics. + +topic_id The topic identifier. + +partitions The data for the partitions. + +partition The partition index. + +state_epoch The state epoch of the share-partition. + +leader_epoch The leader epoch of the share-partition. + +start_offset The share-partition start offset, or -1 if the start offset is not being written. + +state_batches The state batches for the share-partition. + +first_offset The first offset of this state batch. + +last_offset The last offset of this state batch. + +delivery_state The delivery state - 0:Available,2:Acked,4:Archived. + +delivery_count The delivery count. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields
++WriteShareGroupState Response (Version: 0) => [results] _tagged_fields + results => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition error_code error_message _tagged_fields + partition => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +Response header version: 1
++
+Field +Description ++ +results The write results. + +topic_id The topic identifier. + +partitions The results for the partitions. + +partition The partition index. + +error_code The error code, or 0 if there was no error. + +error_message The error message, or null if there was no error. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields DeleteShareGroupState API (Key: 86):
+ +Requests:
++Responses:DeleteShareGroupState Request (Version: 0) => group_id [topics] _tagged_fields + group_id => COMPACT_STRING + topics => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition _tagged_fields + partition => INT32 +Request header version: 2
++
+Field +Description ++ +group_id The group identifier. + +topics The data for the topics. + +topic_id The topic identifier. + +partitions The data for the partitions. + +partition The partition index. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields
++DeleteShareGroupState Response (Version: 0) => [results] _tagged_fields + results => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition error_code error_message _tagged_fields + partition => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +Response header version: 1
++
+Field +Description ++ +results The delete results. + +topic_id The topic identifier. + +partitions The results for the partitions. + +partition The partition index. + +error_code The error code, or 0 if there was no error. + +error_message The error message, or null if there was no error. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields ReadShareGroupStateSummary API (Key: 87):
+ +Requests:
++Responses:ReadShareGroupStateSummary Request (Version: 0) => group_id [topics] _tagged_fields + group_id => COMPACT_STRING + topics => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition leader_epoch _tagged_fields + partition => INT32 + leader_epoch => INT32 +Request header version: 2
++
+Field +Description ++ +group_id The group identifier. + +topics The data for the topics. + +topic_id The topic identifier. + +partitions The data for the partitions. + +partition The partition index. + +leader_epoch The leader epoch of the share-partition. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields
++ReadShareGroupStateSummary Response (Version: 0) => [results] _tagged_fields + results => topic_id [partitions] _tagged_fields + topic_id => UUID + partitions => partition error_code error_message state_epoch leader_epoch start_offset _tagged_fields + partition => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + state_epoch => INT32 + leader_epoch => INT32 + start_offset => INT64 +Response header version: 1
++
+Field +Description ++ +results The read results. + +topic_id The topic identifier. + +partitions The results for the partitions. + +partition The partition index. + +error_code The error code, or 0 if there was no error. + +error_message The error message, or null if there was no error. + +state_epoch The state epoch of the share-partition. + +leader_epoch The leader epoch of the share-partition. + +start_offset The share-partition start offset. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields StreamsGroupHeartbeat API (Key: 88):
+ +Requests:
++Responses:StreamsGroupHeartbeat Request (Version: 0) => group_id member_id member_epoch endpoint_information_epoch instance_id rack_id rebalance_timeout_ms topology [active_tasks] [standby_tasks] [warmup_tasks] process_id user_endpoint [client_tags] [task_offsets] [task_end_offsets] shutdown_application _tagged_fields + group_id => COMPACT_STRING + member_id => COMPACT_STRING + member_epoch => INT32 + endpoint_information_epoch => INT32 + instance_id => COMPACT_NULLABLE_STRING + rack_id => COMPACT_NULLABLE_STRING + rebalance_timeout_ms => INT32 + topology => epoch [subtopologies] _tagged_fields + epoch => INT32 + subtopologies => subtopology_id [source_topics] [source_topic_regex] [state_changelog_topics] [repartition_sink_topics] [repartition_source_topics] [copartition_groups] _tagged_fields + subtopology_id => COMPACT_STRING + source_topics => COMPACT_STRING + source_topic_regex => COMPACT_STRING + state_changelog_topics => name partitions replication_factor [topic_configs] _tagged_fields + name => COMPACT_STRING + partitions => INT32 + replication_factor => INT16 + topic_configs => key value _tagged_fields + key => COMPACT_STRING + value => COMPACT_STRING + repartition_sink_topics => COMPACT_STRING + repartition_source_topics => name partitions replication_factor [topic_configs] _tagged_fields + name => COMPACT_STRING + partitions => INT32 + replication_factor => INT16 + topic_configs => key value _tagged_fields + key => COMPACT_STRING + value => COMPACT_STRING + copartition_groups => [source_topics] [source_topic_regex] [repartition_source_topics] _tagged_fields + source_topics => INT16 + source_topic_regex => INT16 + repartition_source_topics => INT16 + active_tasks => subtopology_id [partitions] _tagged_fields + subtopology_id => COMPACT_STRING + partitions => INT32 + standby_tasks => subtopology_id [partitions] _tagged_fields + subtopology_id => COMPACT_STRING + partitions => INT32 + warmup_tasks => subtopology_id [partitions] _tagged_fields + subtopology_id => COMPACT_STRING + partitions => INT32 + process_id => COMPACT_NULLABLE_STRING + user_endpoint => host port _tagged_fields + host => COMPACT_STRING + port => UINT16 + client_tags => key value _tagged_fields + key => COMPACT_STRING + value => COMPACT_STRING + task_offsets => subtopology_id partition offset _tagged_fields + subtopology_id => COMPACT_STRING + partition => INT32 + offset => INT64 + task_end_offsets => subtopology_id partition offset _tagged_fields + subtopology_id => COMPACT_STRING + partition => INT32 + offset => INT64 + shutdown_application => BOOLEAN +This version of the request is unstable.
Request header version: 2
++
+Field +Description ++ +group_id The group identifier. + +member_id The member ID generated by the streams consumer. The member ID must be kept during the entire lifetime of the streams consumer process. + +member_epoch The current member epoch; 0 to join the group; -1 to leave the group; -2 to indicate that the static member will rejoin. + +endpoint_information_epoch The current endpoint epoch of this client, represents the latest endpoint epoch this client received + +instance_id null if not provided or if it didn't change since the last heartbeat; the instance ID for static membership otherwise. + +rack_id null if not provided or if it didn't change since the last heartbeat; the rack ID of the member otherwise. + +rebalance_timeout_ms -1 if it didn't change since the last heartbeat; the maximum time in milliseconds that the coordinator will wait on the member to revoke its tasks otherwise. + +topology The topology metadata of the streams application. Used to initialize the topology of the group and to check if the topology corresponds to the topology initialized for the group. Only sent when memberEpoch = 0, must be non-empty. Null otherwise. + +epoch The epoch of the topology. Used to check if the topology corresponds to the topology initialized on the brokers. + +subtopologies The sub-topologies of the streams application. + +subtopology_id String to uniquely identify the subtopology. Deterministically generated from the topology + +source_topics The topics the topology reads from. + +source_topic_regex The regular expressions identifying topics the subtopology reads from. + +state_changelog_topics The set of state changelog topics associated with this subtopology. Created automatically. + +name The name of the topic. + +partitions The number of partitions in the topic. Can be 0 if no specific number of partitions is enforced. Always 0 for changelog topics. + +replication_factor The replication factor of the topic. Can be 0 if the default replication factor should be used. + +topic_configs Topic-level configurations as key-value pairs. + +key key of the config + +value value of the config + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +repartition_sink_topics The repartition topics the subtopology writes to. + +repartition_source_topics The set of source topics that are internally created repartition topics. Created automatically. + +copartition_groups A subset of source topics that must be copartitioned. + +source_topics The topics the topology reads from. Index into the array on the subtopology level. + +source_topic_regex Regular expressions identifying topics the subtopology reads from. Index into the array on the subtopology level. + +repartition_source_topics The set of source topics that are internally created repartition topics. Index into the array on the subtopology level. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +active_tasks Currently owned active tasks for this client. Null if unchanged since last heartbeat. + +subtopology_id The subtopology identifier. + +partitions The partitions of the input topics processed by this member. + +_tagged_fields The tagged fields + +standby_tasks Currently owned standby tasks for this client. Null if unchanged since last heartbeat. + +warmup_tasks Currently owned warm-up tasks for this client. Null if unchanged since last heartbeat. + +process_id Identity of the streams instance that may have multiple consumers. Null if unchanged since last heartbeat. + +user_endpoint User-defined endpoint for Interactive Queries. Null if unchanged since last heartbeat, or if not defined on the client. + +host host of the endpoint + +port port of the endpoint + +_tagged_fields The tagged fields + +client_tags Used for rack-aware assignment algorithm. Null if unchanged since last heartbeat. + +task_offsets Cumulative changelog offsets for tasks. Only updated when a warm-up task has caught up, and according to the task offset interval. Null if unchanged since last heartbeat. + +subtopology_id The subtopology identifier. + +partition The partition. + +offset The offset. + +_tagged_fields The tagged fields + +task_end_offsets Cumulative changelog end-offsets for tasks. Only updated when a warm-up task has caught up, and according to the task offset interval. Null if unchanged since last heartbeat. + +shutdown_application Whether all Streams clients in the group should shut down. + +_tagged_fields The tagged fields
++StreamsGroupHeartbeat Response (Version: 0) => throttle_time_ms error_code error_message member_id member_epoch heartbeat_interval_ms acceptable_recovery_lag task_offset_interval_ms [status] [active_tasks] [standby_tasks] [warmup_tasks] endpoint_information_epoch [partitions_by_user_endpoint] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + member_id => COMPACT_STRING + member_epoch => INT32 + heartbeat_interval_ms => INT32 + acceptable_recovery_lag => INT32 + task_offset_interval_ms => INT32 + status => status_code status_detail _tagged_fields + status_code => INT8 + status_detail => COMPACT_STRING + active_tasks => subtopology_id [partitions] _tagged_fields + subtopology_id => COMPACT_STRING + partitions => INT32 + standby_tasks => subtopology_id [partitions] _tagged_fields + subtopology_id => COMPACT_STRING + partitions => INT32 + warmup_tasks => subtopology_id [partitions] _tagged_fields + subtopology_id => COMPACT_STRING + partitions => INT32 + endpoint_information_epoch => INT32 + partitions_by_user_endpoint => user_endpoint [active_partitions] [standby_partitions] _tagged_fields + user_endpoint => host port _tagged_fields + host => COMPACT_STRING + port => UINT16 + active_partitions => topic [partitions] _tagged_fields + topic => COMPACT_STRING + partitions => INT32 + standby_partitions => topic [partitions] _tagged_fields + topic => COMPACT_STRING + partitions => INT32 +Response header version: 1
++
+Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +error_code The top-level error code, or 0 if there was no error + +error_message The top-level error message, or null if there was no error. + +member_id The member id is always generated by the streams consumer. + +member_epoch The member epoch. + +heartbeat_interval_ms The heartbeat interval in milliseconds. + +acceptable_recovery_lag The maximal lag a warm-up task can have to be considered caught-up. + +task_offset_interval_ms The interval in which the task changelog offsets on a client are updated on the broker. The offsets are sent with the next heartbeat after this time has passed. + +status Indicate zero or more status for the group. Null if unchanged since last heartbeat. + +status_code A code to indicate that a particular status is active for the group membership + +status_detail A string representation of the status. + +_tagged_fields The tagged fields + +active_tasks Assigned active tasks for this client. Null if unchanged since last heartbeat. + +subtopology_id The subtopology identifier. + +partitions The partitions of the input topics processed by this member. + +_tagged_fields The tagged fields + +standby_tasks Assigned standby tasks for this client. Null if unchanged since last heartbeat. + +warmup_tasks Assigned warm-up tasks for this client. Null if unchanged since last heartbeat. + +endpoint_information_epoch The endpoint epoch set in the response + +partitions_by_user_endpoint Global assignment information used for IQ. Null if unchanged since last heartbeat. + +user_endpoint User-defined endpoint to connect to the node + +host host of the endpoint + +port port of the endpoint + +_tagged_fields The tagged fields + +active_partitions All topic partitions materialized by active tasks on the node + +topic topic name + +partitions partitions + +_tagged_fields The tagged fields + +standby_partitions All topic partitions materialized by standby tasks on the node + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields StreamsGroupDescribe API (Key: 89):
+ +Requests:
++Responses:StreamsGroupDescribe Request (Version: 0) => [group_ids] include_authorized_operations _tagged_fields + group_ids => COMPACT_STRING + include_authorized_operations => BOOLEAN +This version of the request is unstable.
Request header version: 2
++
+Field +Description ++ +group_ids The ids of the groups to describe + +include_authorized_operations Whether to include authorized operations. + +_tagged_fields The tagged fields
+-Responses:StreamsGroupDescribe Response (Version: 0) => throttle_time_ms [groups] _tagged_fields + throttle_time_ms => INT32 + groups => error_code error_message group_id group_state group_epoch assignment_epoch topology [members] authorized_operations _tagged_fields + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + group_id => COMPACT_STRING + group_state => COMPACT_STRING + group_epoch => INT32 + assignment_epoch => INT32 + topology => epoch [subtopologies] _tagged_fields + epoch => INT32 + subtopologies => subtopology_id [source_topics] [repartition_sink_topics] [state_changelog_topics] [repartition_source_topics] _tagged_fields + subtopology_id => COMPACT_STRING + source_topics => COMPACT_STRING + repartition_sink_topics => COMPACT_STRING + state_changelog_topics => name partitions replication_factor [topic_configs] _tagged_fields + name => COMPACT_STRING + partitions => INT32 + replication_factor => INT16 + topic_configs => key value _tagged_fields + key => COMPACT_STRING + value => COMPACT_STRING + repartition_source_topics => name partitions replication_factor [topic_configs] _tagged_fields + name => COMPACT_STRING + partitions => INT32 + replication_factor => INT16 + topic_configs => key value _tagged_fields + key => COMPACT_STRING + value => COMPACT_STRING + members => member_id member_epoch instance_id rack_id client_id client_host topology_epoch process_id user_endpoint [client_tags] [task_offsets] [task_end_offsets] assignment target_assignment is_classic _tagged_fields + member_id => COMPACT_STRING + member_epoch => INT32 + instance_id => COMPACT_NULLABLE_STRING + rack_id => COMPACT_NULLABLE_STRING + client_id => COMPACT_STRING + client_host => COMPACT_STRING + topology_epoch => INT32 + process_id => COMPACT_STRING + user_endpoint => host port _tagged_fields + host => COMPACT_STRING + port => UINT16 + client_tags => key value _tagged_fields + key => COMPACT_STRING + value => COMPACT_STRING + task_offsets => subtopology_id partition offset _tagged_fields + subtopology_id => COMPACT_STRING + partition => INT32 + offset => INT64 + task_end_offsets => subtopology_id partition offset _tagged_fields + subtopology_id => COMPACT_STRING + partition => INT32 + offset => INT64 + assignment => [active_tasks] [standby_tasks] [warmup_tasks] _tagged_fields + active_tasks => subtopology_id [partitions] _tagged_fields + subtopology_id => COMPACT_STRING + partitions => INT32 + standby_tasks => subtopology_id [partitions] _tagged_fields + subtopology_id => COMPACT_STRING + partitions => INT32 + warmup_tasks => subtopology_id [partitions] _tagged_fields + subtopology_id => COMPACT_STRING + partitions => INT32 + target_assignment => [active_tasks] [standby_tasks] [warmup_tasks] _tagged_fields + active_tasks => subtopology_id [partitions] _tagged_fields + subtopology_id => COMPACT_STRING + partitions => INT32 + standby_tasks => subtopology_id [partitions] _tagged_fields + subtopology_id => COMPACT_STRING + partitions => INT32 + warmup_tasks => subtopology_id [partitions] _tagged_fields + subtopology_id => COMPACT_STRING + partitions => INT32 + is_classic => BOOLEAN + authorized_operations => INT32 +Response header version: 1
++
-Field +Description ++ +throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + +groups Each described group. + +error_code The describe error, or 0 if there was no error. + +error_message The top-level error message, or null if there was no error. + +group_id The group ID string. + +group_state The group state string, or the empty string. + +group_epoch The group epoch. + +assignment_epoch The assignment epoch. + +topology The topology metadata currently initialized for the streams application. Can be null in case of a describe error. + +epoch The epoch of the currently initialized topology for this group. + +subtopologies The subtopologies of the streams application. This contains the configured subtopologies, where the number of partitions are set and any regular expressions are resolved to actual topics. Null if the group is uninitialized, source topics are missing or incorrectly partitioned. + +subtopology_id String to uniquely identify the subtopology. + +source_topics The topics the subtopology reads from. + +repartition_sink_topics The repartition topics the subtopology writes to. + +state_changelog_topics The set of state changelog topics associated with this subtopology. Created automatically. + +name The name of the topic. + +partitions The number of partitions in the topic. Can be 0 if no specific number of partitions is enforced. Always 0 for changelog topics. + +replication_factor The replication factor of the topic. Can be 0 if the default replication factor should be used. + +topic_configs Topic-level configurations as key-value pairs. + +key key of the config + +value value of the config + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +repartition_source_topics The set of source topics that are internally created repartition topics. Created automatically. + +_tagged_fields The tagged fields + +_tagged_fields The tagged fields + +members The members. + +member_id The member ID. + +member_epoch The member epoch. + +instance_id The member instance ID for static membership. + +rack_id The rack ID. + +client_id The client ID. + +client_host The client host. + topology_epoch The epoch of the topology on the client. - +acknowledgement_batches Record batches to acknowledge. process_id Identity of the streams instance that may have multiple clients. - +first_offset First offset of batch of records to acknowledge. user_endpoint User-defined endpoint for Interactive Queries. Null if not defined for this client. - +last_offset Last offset (inclusive) of batch of records to acknowledge. host host of the endpoint - +acknowledge_types Array of acknowledge types - 0:Gap,1:Accept,2:Release,3:Reject. port port of the endpoint _tagged_fields The tagged fields - +_tagged_fields The tagged fields client_tags Used for rack-aware assignment algorithm. - +_tagged_fields The tagged fields task_offsets Cumulative changelog offsets for tasks. - +forgotten_topics_data The partitions to remove from this share session. subtopology_id The subtopology identifier. - +topic_id The unique topic ID. partition The partition. - +partitions The partitions indexes to forget. offset The offset. _tagged_fields The tagged fields - -_tagged_fields The tagged fields
-ShareAcknowledge API (Key: 79):
- -Requests:
--Responses:ShareAcknowledge Request (Version: 0) => group_id member_id share_session_epoch [topics] _tagged_fields - group_id => COMPACT_NULLABLE_STRING - member_id => COMPACT_NULLABLE_STRING - share_session_epoch => INT32 - topics => topic_id [partitions] _tagged_fields - topic_id => UUID - partitions => partition_index [acknowledgement_batches] _tagged_fields - partition_index => INT32 - acknowledgement_batches => first_offset last_offset [acknowledge_types] _tagged_fields - first_offset => INT64 - last_offset => INT64 - acknowledge_types => INT8 -This version of the request is unstable.
Request header version: 2
--
Field -Description -- +group_id The group identifier. task_end_offsets Cumulative changelog end offsets for tasks. - +member_id The member ID. assignment The current assignment. - +share_session_epoch The current share session epoch: 0 to open a share session; -1 to close it; otherwise increments for consecutive requests. active_tasks Active tasks for this client. - +topics The topics containing records to acknowledge. subtopology_id The subtopology identifier. - +topic_id The unique topic ID. partitions The partitions of the input topics processed by this member. - +partitions The partitions containing records to acknowledge. _tagged_fields The tagged fields - +partition_index The partition index. standby_tasks Standby tasks for this client. - +acknowledgement_batches Record batches to acknowledge. warmup_tasks Warm-up tasks for this client. - +first_offset First offset of batch of records to acknowledge. _tagged_fields The tagged fields - +last_offset Last offset (inclusive) of batch of records to acknowledge. target_assignment The target assignment. - +acknowledge_types Array of acknowledge types - 0:Gap,1:Accept,2:Release,3:Reject. is_classic True for classic members that have not been upgraded yet. _tagged_fields The tagged fields - +_tagged_fields The tagged fields authorized_operations 32-bit bitfield to represent authorized operations for this group. _tagged_fields The tagged fields _tagged_fields The tagged fields
-AddRaftVoter API (Key: 80):
+DescribeShareGroupOffsets API (Key: 90):
Requests:
-AddRaftVoter Request (Version: 0) => cluster_id timeout_ms voter_id voter_directory_id [listeners] _tagged_fields - cluster_id => COMPACT_NULLABLE_STRING - timeout_ms => INT32 - voter_id => INT32 - voter_directory_id => UUID - listeners => name host port _tagged_fields - name => COMPACT_STRING - host => COMPACT_STRING - port => UINT16 +Responses:DescribeShareGroupOffsets Request (Version: 0) => [groups] _tagged_fields + groups => group_id [topics] _tagged_fields + group_id => COMPACT_STRING + topics => topic_name [partitions] _tagged_fields + topic_name => COMPACT_STRING + partitions => INT32Request header version: 2
Field Description - +cluster_id The cluster id. groups The groups to describe offsets for. - -timeout_ms The maximum time to wait for the request to complete before returning. - -voter_id The replica id of the voter getting added to the topic partition. - +voter_directory_id The directory id of the voter getting added to the topic partition. group_id The group identifier. - +listeners The endpoints that can be used to communicate with the voter. topics The topics to describe offsets for, or null for all topic-partitions. - +name The name of the endpoint. topic_name The topic name. - +host The hostname. partitions The partitions. - +port The port. _tagged_fields The tagged fields _tagged_fields The tagged fields @@ -13850,92 +18149,89 @@ AddRaftVoter API (Key: 80):
-RemoveRaftVoter API (Key: 81):
- -Requests:
-RemoveRaftVoter Request (Version: 0) => cluster_id voter_id voter_directory_id _tagged_fields - cluster_id => COMPACT_NULLABLE_STRING - voter_id => INT32 - voter_directory_id => UUID -Request header version: 2
+-Responses:DescribeShareGroupOffsets Response (Version: 0) => throttle_time_ms [groups] _tagged_fields + throttle_time_ms => INT32 + groups => group_id [topics] error_code error_message _tagged_fields + group_id => COMPACT_STRING + topics => topic_name topic_id [partitions] _tagged_fields + topic_name => COMPACT_STRING + topic_id => UUID + partitions => partition_index start_offset leader_epoch error_code error_message _tagged_fields + partition_index => INT32 + start_offset => INT64 + leader_epoch => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +Response header version: 1
-
Field Description - -cluster_id The cluster id of the request. - +voter_id The replica id of the voter getting removed from the topic partition. throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. - +voter_directory_id The directory id of the voter getting removed from the topic partition. groups The results for each group. - -_tagged_fields The tagged fields
-InitializeShareGroupState API (Key: 83):
- -Requests:
--Responses:InitializeShareGroupState Request (Version: 0) => group_id [topics] _tagged_fields - group_id => COMPACT_STRING - topics => topic_id [partitions] _tagged_fields - topic_id => UUID - partitions => partition state_epoch start_offset _tagged_fields - partition => INT32 - state_epoch => INT32 - start_offset => INT64 -This version of the request is unstable.
Request header version: 2
--
Field -Description -group_id The group identifier. - +topics The data for the topics. topics The results for each topic. - +topic_id The topic identifier. topic_name The topic name. - +partitions The data for the partitions. topic_id The unique topic ID. - +partition The partition index. partitions - +state_epoch The state epoch for this share-partition. partition_index The partition index. - +start_offset The share-partition start offset, or -1 if the start offset is not being initialized. start_offset The share-partition start offset. ++ +leader_epoch The leader epoch of the partition. + +error_code The partition-level error code, or 0 if there was no error. + error_message The partition-level error message, or null if there was no error. _tagged_fields The tagged fields _tagged_fields The tagged fields + +error_code The group-level error code, or 0 if there was no error. + +error_message The group-level error message, or null if there was no error. + +_tagged_fields The tagged fields _tagged_fields The tagged fields
-ReadShareGroupState API (Key: 84):
+AlterShareGroupOffsets API (Key: 91):
Requests:
-ReadShareGroupState Request (Version: 0) => group_id [topics] _tagged_fields +Responses:AlterShareGroupOffsets Request (Version: 0) => group_id [topics] _tagged_fields group_id => COMPACT_STRING - topics => topic_id [partitions] _tagged_fields - topic_id => UUID - partitions => partition leader_epoch _tagged_fields - partition => INT32 - leader_epoch => INT32 -This version of the request is unstable.
Request header version: 2
+ topics => topic_name [partitions] _tagged_fields + topic_name => COMPACT_STRING + partitions => partition_index start_offset _tagged_fields + partition_index => INT32 + start_offset => INT64 +Request header version: 2
Field Description group_id The group identifier. - +topics The data for the topics. topics The topics to alter offsets for. - +topic_id The topic identifier. topic_name The topic name. - +partitions The data for the partitions. partitions Each partition to alter offsets for. - +partition The partition index. partition_index The partition index. - +leader_epoch The leader epoch of the share-partition. start_offset The share-partition start offset. _tagged_fields The tagged fields @@ -13945,55 +18241,35 @@ ReadShareGroupState API (Key: 84)
-WriteShareGroupState API (Key: 85):
- -Requests:
-WriteShareGroupState Request (Version: 0) => group_id [topics] _tagged_fields - group_id => COMPACT_STRING - topics => topic_id [partitions] _tagged_fields +-Responses:AlterShareGroupOffsets Response (Version: 0) => throttle_time_ms [responses] _tagged_fields + throttle_time_ms => INT32 + responses => topic_name topic_id [partitions] _tagged_fields + topic_name => COMPACT_STRING topic_id => UUID - partitions => partition state_epoch leader_epoch start_offset [state_batches] _tagged_fields - partition => INT32 - state_epoch => INT32 - leader_epoch => INT32 - start_offset => INT64 - state_batches => first_offset last_offset delivery_state delivery_count _tagged_fields - first_offset => INT64 - last_offset => INT64 - delivery_state => INT8 - delivery_count => INT16 -This version of the request is unstable.
Request header version: 2
+ partitions => partition_index error_code error_message _tagged_fields + partition_index => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +Response header version: 1
Field Description - -group_id The group identifier. - -topics The data for the topics. - -topic_id The topic identifier. - -partitions The data for the partitions. - -partition The partition index. - -state_epoch The state epoch for this share-partition. - +leader_epoch The leader epoch of the share-partition. throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. - +start_offset The share-partition start offset, or -1 if the start offset is not being written. responses The results for each topic. - +state_batches The state batches for the share-partition. topic_name The topic name. - +first_offset The base offset of this state batch. topic_id The unique topic ID. - +last_offset The last offset of this state batch. partitions - +delivery_state The state - 0:Available,2:Acked,4:Archived. partition_index The partition index. - +delivery_count The delivery count. error_code The error code, or 0 if there was no error. - +_tagged_fields The tagged fields error_message The error message, or null if there was no error. _tagged_fields The tagged fields @@ -14002,32 +18278,23 @@ WriteShareGroupState API (Key: 8
_tagged_fields The tagged fields
-DeleteShareGroupState API (Key: 86):
+DeleteShareGroupOffsets API (Key: 92):
Requests:
-DeleteShareGroupState Request (Version: 0) => group_id [topics] _tagged_fields +Responses:DeleteShareGroupOffsets Request (Version: 0) => group_id [topics] _tagged_fields group_id => COMPACT_STRING - topics => topic_id [partitions] _tagged_fields - topic_id => UUID - partitions => partition _tagged_fields - partition => INT32 -This version of the request is unstable.
Request header version: 2
+ topics => topic_name _tagged_fields + topic_name => COMPACT_STRING +Request header version: 2
Field Description group_id The group identifier. - -topics The data for the topics. - -topic_id The topic identifier. - -partitions The data for the partitions. - +partition The partition index. topics The topics to delete offsets for. - +_tagged_fields The tagged fields topic_name The topic name. _tagged_fields The tagged fields @@ -14035,39 +18302,39 @@ DeleteShareGroupState API (Key:
-ReadShareGroupStateSummary API (Key: 87):
- -Requests:
-ReadShareGroupStateSummary Request (Version: 0) => group_id [topics] _tagged_fields - group_id => COMPACT_STRING - topics => topic_id [partitions] _tagged_fields +-Responses:DeleteShareGroupOffsets Response (Version: 0) => throttle_time_ms error_code error_message [responses] _tagged_fields + throttle_time_ms => INT32 + error_code => INT16 + error_message => COMPACT_NULLABLE_STRING + responses => topic_name topic_id error_code error_message _tagged_fields + topic_name => COMPACT_STRING topic_id => UUID - partitions => partition leader_epoch _tagged_fields - partition => INT32 - leader_epoch => INT32 -This version of the request is unstable.
Request header version: 2
+ error_code => INT16 + error_message => COMPACT_NULLABLE_STRING +Response header version: 1
Field Description - +group_id The group identifier. throttle_time_ms The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. - +topics The data for the topics. error_code The top-level error code, or 0 if there was no error. - +topic_id The topic identifier. error_message The top-level error message, or null if there was no error. - +partitions The data for the partitions. responses The results for each topic. - +partition The partition index. topic_name The topic name. - +leader_epoch The leader epoch of the share-partition. topic_id The unique topic ID. - +_tagged_fields The tagged fields error_code The topic-level error code, or 0 if there was no error. ++ error_message The topic-level error message, or null if there was no error. _tagged_fields The tagged fields _tagged_fields The tagged fields