blob: b22a9a04ed11a08cef661f1852917f78d7bff249 [file] [log] [blame]
<h5>Headers:</h5>
<pre class="line-numbers"><code class="language-java">Request Header v0 => request_api_key request_api_version correlation_id
request_api_key => INT16
request_api_version => INT16
correlation_id => INT32
</code></pre>
<table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>request_api_key</td><td>The API key of this request.</td></tr>
<tr>
<td>request_api_version</td><td>The API version of this request.</td></tr>
<tr>
<td>correlation_id</td><td>The correlation ID of this request.</td></tr>
</table>
<pre class="line-numbers"><code class="language-java">Request Header v1 => request_api_key request_api_version correlation_id client_id
request_api_key => INT16
request_api_version => INT16
correlation_id => INT32
client_id => NULLABLE_STRING
</code></pre>
<table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>request_api_key</td><td>The API key of this request.</td></tr>
<tr>
<td>request_api_version</td><td>The API version of this request.</td></tr>
<tr>
<td>correlation_id</td><td>The correlation ID of this request.</td></tr>
<tr>
<td>client_id</td><td>The client ID string.</td></tr>
</table>
<pre class="line-numbers"><code class="language-java">Request Header v2 => request_api_key request_api_version correlation_id client_id TAG_BUFFER
request_api_key => INT16
request_api_version => INT16
correlation_id => INT32
client_id => NULLABLE_STRING
</code></pre>
<table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>request_api_key</td><td>The API key of this request.</td></tr>
<tr>
<td>request_api_version</td><td>The API version of this request.</td></tr>
<tr>
<td>correlation_id</td><td>The correlation ID of this request.</td></tr>
<tr>
<td>client_id</td><td>The client ID string.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
<pre class="line-numbers"><code class="language-java">Response Header v0 => correlation_id
correlation_id => INT32
</code></pre>
<table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>correlation_id</td><td>The correlation ID of this response.</td></tr>
</table>
<pre class="line-numbers"><code class="language-java">Response Header v1 => correlation_id TAG_BUFFER
correlation_id => INT32
</code></pre>
<table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>correlation_id</td><td>The correlation ID of this response.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
<h5><a name="The_Messages_Produce">Produce API (Key: 0):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">Produce Request (Version: 0) => acks timeout [topic_data]
acks => INT16
timeout => INT32
topic_data => topic [data]
topic => STRING
data => partition record_set
partition => INT32
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>acks</td><td>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.</td></tr>
<tr>
<td>timeout</td><td>The time to await a response in ms.</td></tr>
<tr>
<td>topic_data</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>data</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Produce Request (Version: 1) => acks timeout [topic_data]
acks => INT16
timeout => INT32
topic_data => topic [data]
topic => STRING
data => partition record_set
partition => INT32
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>acks</td><td>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.</td></tr>
<tr>
<td>timeout</td><td>The time to await a response in ms.</td></tr>
<tr>
<td>topic_data</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>data</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Produce Request (Version: 2) => acks timeout [topic_data]
acks => INT16
timeout => INT32
topic_data => topic [data]
topic => STRING
data => partition record_set
partition => INT32
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>acks</td><td>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.</td></tr>
<tr>
<td>timeout</td><td>The time to await a response in ms.</td></tr>
<tr>
<td>topic_data</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>data</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Produce Request (Version: 3) => transactional_id acks timeout [topic_data]
transactional_id => NULLABLE_STRING
acks => INT16
timeout => INT32
topic_data => topic [data]
topic => STRING
data => partition record_set
partition => INT32
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The transactional id or null if the producer is not transactional</td></tr>
<tr>
<td>acks</td><td>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.</td></tr>
<tr>
<td>timeout</td><td>The time to await a response in ms.</td></tr>
<tr>
<td>topic_data</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>data</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Produce Request (Version: 4) => transactional_id acks timeout [topic_data]
transactional_id => NULLABLE_STRING
acks => INT16
timeout => INT32
topic_data => topic [data]
topic => STRING
data => partition record_set
partition => INT32
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The transactional id or null if the producer is not transactional</td></tr>
<tr>
<td>acks</td><td>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.</td></tr>
<tr>
<td>timeout</td><td>The time to await a response in ms.</td></tr>
<tr>
<td>topic_data</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>data</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Produce Request (Version: 5) => transactional_id acks timeout [topic_data]
transactional_id => NULLABLE_STRING
acks => INT16
timeout => INT32
topic_data => topic [data]
topic => STRING
data => partition record_set
partition => INT32
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The transactional id or null if the producer is not transactional</td></tr>
<tr>
<td>acks</td><td>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.</td></tr>
<tr>
<td>timeout</td><td>The time to await a response in ms.</td></tr>
<tr>
<td>topic_data</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>data</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Produce Request (Version: 6) => transactional_id acks timeout [topic_data]
transactional_id => NULLABLE_STRING
acks => INT16
timeout => INT32
topic_data => topic [data]
topic => STRING
data => partition record_set
partition => INT32
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The transactional id or null if the producer is not transactional</td></tr>
<tr>
<td>acks</td><td>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.</td></tr>
<tr>
<td>timeout</td><td>The time to await a response in ms.</td></tr>
<tr>
<td>topic_data</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>data</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Produce Request (Version: 7) => transactional_id acks timeout [topic_data]
transactional_id => NULLABLE_STRING
acks => INT16
timeout => INT32
topic_data => topic [data]
topic => STRING
data => partition record_set
partition => INT32
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The transactional id or null if the producer is not transactional</td></tr>
<tr>
<td>acks</td><td>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.</td></tr>
<tr>
<td>timeout</td><td>The time to await a response in ms.</td></tr>
<tr>
<td>topic_data</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>data</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Produce Request (Version: 8) => transactional_id acks timeout [topic_data]
transactional_id => NULLABLE_STRING
acks => INT16
timeout => INT32
topic_data => topic [data]
topic => STRING
data => partition record_set
partition => INT32
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The transactional id or null if the producer is not transactional</td></tr>
<tr>
<td>acks</td><td>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.</td></tr>
<tr>
<td>timeout</td><td>The time to await a response in ms.</td></tr>
<tr>
<td>topic_data</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>data</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">Produce Response (Version: 0) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code base_offset
partition => INT32
error_code => INT16
base_offset => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>base_offset</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Produce Response (Version: 1) => [responses] throttle_time_ms
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code base_offset
partition => INT32
error_code => INT16
base_offset => INT64
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>base_offset</td><td>null</td></tr>
<tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Produce Response (Version: 2) => [responses] throttle_time_ms
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code base_offset log_append_time
partition => INT32
error_code => INT16
base_offset => INT64
log_append_time => INT64
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>base_offset</td><td>null</td></tr>
<tr>
<td>log_append_time</td><td>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.</td></tr>
<tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Produce Response (Version: 3) => [responses] throttle_time_ms
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code base_offset log_append_time
partition => INT32
error_code => INT16
base_offset => INT64
log_append_time => INT64
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>base_offset</td><td>null</td></tr>
<tr>
<td>log_append_time</td><td>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.</td></tr>
<tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Produce Response (Version: 4) => [responses] throttle_time_ms
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code base_offset log_append_time
partition => INT32
error_code => INT16
base_offset => INT64
log_append_time => INT64
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>base_offset</td><td>null</td></tr>
<tr>
<td>log_append_time</td><td>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.</td></tr>
<tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Produce Response (Version: 5) => [responses] throttle_time_ms
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code base_offset log_append_time log_start_offset
partition => INT32
error_code => INT16
base_offset => INT64
log_append_time => INT64
log_start_offset => INT64
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>base_offset</td><td>null</td></tr>
<tr>
<td>log_append_time</td><td>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.</td></tr>
<tr>
<td>log_start_offset</td><td>The start offset of the log at the time this produce response was created</td></tr>
<tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Produce Response (Version: 6) => [responses] throttle_time_ms
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code base_offset log_append_time log_start_offset
partition => INT32
error_code => INT16
base_offset => INT64
log_append_time => INT64
log_start_offset => INT64
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>base_offset</td><td>null</td></tr>
<tr>
<td>log_append_time</td><td>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.</td></tr>
<tr>
<td>log_start_offset</td><td>The start offset of the log at the time this produce response was created</td></tr>
<tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Produce Response (Version: 7) => [responses] throttle_time_ms
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code base_offset log_append_time log_start_offset
partition => INT32
error_code => INT16
base_offset => INT64
log_append_time => INT64
log_start_offset => INT64
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>base_offset</td><td>null</td></tr>
<tr>
<td>log_append_time</td><td>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.</td></tr>
<tr>
<td>log_start_offset</td><td>The start offset of the log at the time this produce response was created</td></tr>
<tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Produce Response (Version: 8) => [responses] throttle_time_ms
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code base_offset log_append_time log_start_offset [record_errors] error_message
partition => INT32
error_code => INT16
base_offset => INT64
log_append_time => INT64
log_start_offset => INT64
record_errors => batch_index batch_index_error_message
batch_index => INT32
batch_index_error_message => NULLABLE_STRING
error_message => NULLABLE_STRING
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>base_offset</td><td>null</td></tr>
<tr>
<td>log_append_time</td><td>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.</td></tr>
<tr>
<td>log_start_offset</td><td>The start offset of the log at the time this produce response was created</td></tr>
<tr>
<td>record_errors</td><td>The batch indices of records that caused the batch to be dropped</td></tr>
<tr>
<td>batch_index</td><td>The batch index of the record that caused the batch to be dropped</td></tr>
<tr>
<td>batch_index_error_message</td><td>The error message of the record that caused the batch to be dropped</td></tr>
<tr>
<td>error_message</td><td>The global error message summarizing the common root cause of the records that caused the batch to be dropped</td></tr>
<tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
</table>
</p>
<h5><a name="The_Messages_Fetch">Fetch API (Key: 1):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">Fetch Request (Version: 0) => replica_id max_wait_time min_bytes [topics]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset partition_max_bytes
partition => INT32
fetch_offset => INT64
partition_max_bytes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>max_wait_time</td><td>Maximum time in ms to wait for the response.</td></tr>
<tr>
<td>min_bytes</td><td>Minimum bytes to accumulate in the response.</td></tr>
<tr>
<td>topics</td><td>Topics to fetch in the order provided.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to fetch.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>fetch_offset</td><td>Message offset.</td></tr>
<tr>
<td>partition_max_bytes</td><td>Maximum bytes to fetch.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Request (Version: 1) => replica_id max_wait_time min_bytes [topics]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset partition_max_bytes
partition => INT32
fetch_offset => INT64
partition_max_bytes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>max_wait_time</td><td>Maximum time in ms to wait for the response.</td></tr>
<tr>
<td>min_bytes</td><td>Minimum bytes to accumulate in the response.</td></tr>
<tr>
<td>topics</td><td>Topics to fetch in the order provided.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to fetch.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>fetch_offset</td><td>Message offset.</td></tr>
<tr>
<td>partition_max_bytes</td><td>Maximum bytes to fetch.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Request (Version: 2) => replica_id max_wait_time min_bytes [topics]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset partition_max_bytes
partition => INT32
fetch_offset => INT64
partition_max_bytes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>max_wait_time</td><td>Maximum time in ms to wait for the response.</td></tr>
<tr>
<td>min_bytes</td><td>Minimum bytes to accumulate in the response.</td></tr>
<tr>
<td>topics</td><td>Topics to fetch in the order provided.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to fetch.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>fetch_offset</td><td>Message offset.</td></tr>
<tr>
<td>partition_max_bytes</td><td>Maximum bytes to fetch.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Request (Version: 3) => replica_id max_wait_time min_bytes max_bytes [topics]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
max_bytes => INT32
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset partition_max_bytes
partition => INT32
fetch_offset => INT64
partition_max_bytes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>max_wait_time</td><td>Maximum time in ms to wait for the response.</td></tr>
<tr>
<td>min_bytes</td><td>Minimum bytes to accumulate in the response.</td></tr>
<tr>
<td>max_bytes</td><td>Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.</td></tr>
<tr>
<td>topics</td><td>Topics to fetch in the order provided.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to fetch.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>fetch_offset</td><td>Message offset.</td></tr>
<tr>
<td>partition_max_bytes</td><td>Maximum bytes to fetch.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Request (Version: 4) => replica_id max_wait_time min_bytes max_bytes isolation_level [topics]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
max_bytes => INT32
isolation_level => INT8
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset partition_max_bytes
partition => INT32
fetch_offset => INT64
partition_max_bytes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>max_wait_time</td><td>Maximum time in ms to wait for the response.</td></tr>
<tr>
<td>min_bytes</td><td>Minimum bytes to accumulate in the response.</td></tr>
<tr>
<td>max_bytes</td><td>Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.</td></tr>
<tr>
<td>isolation_level</td><td>This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records</td></tr>
<tr>
<td>topics</td><td>Topics to fetch in the order provided.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to fetch.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>fetch_offset</td><td>Message offset.</td></tr>
<tr>
<td>partition_max_bytes</td><td>Maximum bytes to fetch.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Request (Version: 5) => replica_id max_wait_time min_bytes max_bytes isolation_level [topics]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
max_bytes => INT32
isolation_level => INT8
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset log_start_offset partition_max_bytes
partition => INT32
fetch_offset => INT64
log_start_offset => INT64
partition_max_bytes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>max_wait_time</td><td>Maximum time in ms to wait for the response.</td></tr>
<tr>
<td>min_bytes</td><td>Minimum bytes to accumulate in the response.</td></tr>
<tr>
<td>max_bytes</td><td>Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.</td></tr>
<tr>
<td>isolation_level</td><td>This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records</td></tr>
<tr>
<td>topics</td><td>Topics to fetch in the order provided.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to fetch.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>fetch_offset</td><td>Message offset.</td></tr>
<tr>
<td>log_start_offset</td><td>Earliest available offset of the follower replica. The field is only used when request is sent by follower. </td></tr>
<tr>
<td>partition_max_bytes</td><td>Maximum bytes to fetch.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Request (Version: 6) => replica_id max_wait_time min_bytes max_bytes isolation_level [topics]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
max_bytes => INT32
isolation_level => INT8
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset log_start_offset partition_max_bytes
partition => INT32
fetch_offset => INT64
log_start_offset => INT64
partition_max_bytes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>max_wait_time</td><td>Maximum time in ms to wait for the response.</td></tr>
<tr>
<td>min_bytes</td><td>Minimum bytes to accumulate in the response.</td></tr>
<tr>
<td>max_bytes</td><td>Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.</td></tr>
<tr>
<td>isolation_level</td><td>This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records</td></tr>
<tr>
<td>topics</td><td>Topics to fetch in the order provided.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to fetch.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>fetch_offset</td><td>Message offset.</td></tr>
<tr>
<td>log_start_offset</td><td>Earliest available offset of the follower replica. The field is only used when request is sent by follower. </td></tr>
<tr>
<td>partition_max_bytes</td><td>Maximum bytes to fetch.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Request (Version: 7) => replica_id max_wait_time min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
max_bytes => INT32
isolation_level => INT8
session_id => INT32
session_epoch => INT32
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset log_start_offset partition_max_bytes
partition => INT32
fetch_offset => INT64
log_start_offset => INT64
partition_max_bytes => INT32
forgotten_topics_data => topic [partitions]
topic => STRING
partitions => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>max_wait_time</td><td>Maximum time in ms to wait for the response.</td></tr>
<tr>
<td>min_bytes</td><td>Minimum bytes to accumulate in the response.</td></tr>
<tr>
<td>max_bytes</td><td>Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.</td></tr>
<tr>
<td>isolation_level</td><td>This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records</td></tr>
<tr>
<td>session_id</td><td>The fetch session ID</td></tr>
<tr>
<td>session_epoch</td><td>The fetch session epoch</td></tr>
<tr>
<td>topics</td><td>Topics to fetch in the order provided.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to fetch.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>fetch_offset</td><td>Message offset.</td></tr>
<tr>
<td>log_start_offset</td><td>Earliest available offset of the follower replica. The field is only used when request is sent by follower. </td></tr>
<tr>
<td>partition_max_bytes</td><td>Maximum bytes to fetch.</td></tr>
<tr>
<td>forgotten_topics_data</td><td>Topics to remove from the fetch session.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to remove from the fetch session.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Request (Version: 8) => replica_id max_wait_time min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
max_bytes => INT32
isolation_level => INT8
session_id => INT32
session_epoch => INT32
topics => topic [partitions]
topic => STRING
partitions => partition fetch_offset log_start_offset partition_max_bytes
partition => INT32
fetch_offset => INT64
log_start_offset => INT64
partition_max_bytes => INT32
forgotten_topics_data => topic [partitions]
topic => STRING
partitions => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>max_wait_time</td><td>Maximum time in ms to wait for the response.</td></tr>
<tr>
<td>min_bytes</td><td>Minimum bytes to accumulate in the response.</td></tr>
<tr>
<td>max_bytes</td><td>Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.</td></tr>
<tr>
<td>isolation_level</td><td>This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records</td></tr>
<tr>
<td>session_id</td><td>The fetch session ID</td></tr>
<tr>
<td>session_epoch</td><td>The fetch session epoch</td></tr>
<tr>
<td>topics</td><td>Topics to fetch in the order provided.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to fetch.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>fetch_offset</td><td>Message offset.</td></tr>
<tr>
<td>log_start_offset</td><td>Earliest available offset of the follower replica. The field is only used when request is sent by follower. </td></tr>
<tr>
<td>partition_max_bytes</td><td>Maximum bytes to fetch.</td></tr>
<tr>
<td>forgotten_topics_data</td><td>Topics to remove from the fetch session.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to remove from the fetch session.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Request (Version: 9) => replica_id max_wait_time min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
max_bytes => INT32
isolation_level => INT8
session_id => INT32
session_epoch => INT32
topics => topic [partitions]
topic => STRING
partitions => partition current_leader_epoch fetch_offset log_start_offset partition_max_bytes
partition => INT32
current_leader_epoch => INT32
fetch_offset => INT64
log_start_offset => INT64
partition_max_bytes => INT32
forgotten_topics_data => topic [partitions]
topic => STRING
partitions => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>max_wait_time</td><td>Maximum time in ms to wait for the response.</td></tr>
<tr>
<td>min_bytes</td><td>Minimum bytes to accumulate in the response.</td></tr>
<tr>
<td>max_bytes</td><td>Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.</td></tr>
<tr>
<td>isolation_level</td><td>This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records</td></tr>
<tr>
<td>session_id</td><td>The fetch session ID</td></tr>
<tr>
<td>session_epoch</td><td>The fetch session epoch</td></tr>
<tr>
<td>topics</td><td>Topics to fetch in the order provided.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to fetch.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>current_leader_epoch</td><td>The current leader epoch, if provided, is used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned.</td></tr>
<tr>
<td>fetch_offset</td><td>Message offset.</td></tr>
<tr>
<td>log_start_offset</td><td>Earliest available offset of the follower replica. The field is only used when request is sent by follower. </td></tr>
<tr>
<td>partition_max_bytes</td><td>Maximum bytes to fetch.</td></tr>
<tr>
<td>forgotten_topics_data</td><td>Topics to remove from the fetch session.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to remove from the fetch session.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Request (Version: 10) => replica_id max_wait_time min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data]
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
max_bytes => INT32
isolation_level => INT8
session_id => INT32
session_epoch => INT32
topics => topic [partitions]
topic => STRING
partitions => partition current_leader_epoch fetch_offset log_start_offset partition_max_bytes
partition => INT32
current_leader_epoch => INT32
fetch_offset => INT64
log_start_offset => INT64
partition_max_bytes => INT32
forgotten_topics_data => topic [partitions]
topic => STRING
partitions => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>max_wait_time</td><td>Maximum time in ms to wait for the response.</td></tr>
<tr>
<td>min_bytes</td><td>Minimum bytes to accumulate in the response.</td></tr>
<tr>
<td>max_bytes</td><td>Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.</td></tr>
<tr>
<td>isolation_level</td><td>This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records</td></tr>
<tr>
<td>session_id</td><td>The fetch session ID</td></tr>
<tr>
<td>session_epoch</td><td>The fetch session epoch</td></tr>
<tr>
<td>topics</td><td>Topics to fetch in the order provided.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to fetch.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>current_leader_epoch</td><td>The current leader epoch, if provided, is used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned.</td></tr>
<tr>
<td>fetch_offset</td><td>Message offset.</td></tr>
<tr>
<td>log_start_offset</td><td>Earliest available offset of the follower replica. The field is only used when request is sent by follower. </td></tr>
<tr>
<td>partition_max_bytes</td><td>Maximum bytes to fetch.</td></tr>
<tr>
<td>forgotten_topics_data</td><td>Topics to remove from the fetch session.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to remove from the fetch session.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Request (Version: 11) => replica_id max_wait_time min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] rack_id
replica_id => INT32
max_wait_time => INT32
min_bytes => INT32
max_bytes => INT32
isolation_level => INT8
session_id => INT32
session_epoch => INT32
topics => topic [partitions]
topic => STRING
partitions => partition current_leader_epoch fetch_offset log_start_offset partition_max_bytes
partition => INT32
current_leader_epoch => INT32
fetch_offset => INT64
log_start_offset => INT64
partition_max_bytes => INT32
forgotten_topics_data => topic [partitions]
topic => STRING
partitions => INT32
rack_id => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>max_wait_time</td><td>Maximum time in ms to wait for the response.</td></tr>
<tr>
<td>min_bytes</td><td>Minimum bytes to accumulate in the response.</td></tr>
<tr>
<td>max_bytes</td><td>Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that progress can be made.</td></tr>
<tr>
<td>isolation_level</td><td>This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records</td></tr>
<tr>
<td>session_id</td><td>The fetch session ID</td></tr>
<tr>
<td>session_epoch</td><td>The fetch session epoch</td></tr>
<tr>
<td>topics</td><td>Topics to fetch in the order provided.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to fetch.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>current_leader_epoch</td><td>The current leader epoch, if provided, is used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned.</td></tr>
<tr>
<td>fetch_offset</td><td>Message offset.</td></tr>
<tr>
<td>log_start_offset</td><td>Earliest available offset of the follower replica. The field is only used when request is sent by follower. </td></tr>
<tr>
<td>partition_max_bytes</td><td>Maximum bytes to fetch.</td></tr>
<tr>
<td>forgotten_topics_data</td><td>Topics to remove from the fetch session.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to remove from the fetch session.</td></tr>
<tr>
<td>rack_id</td><td>The consumer's rack id</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">Fetch Response (Version: 0) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark
partition => INT32
error_code => INT16
high_watermark => INT64
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition_header</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>high_watermark</td><td>Last committed offset.</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Response (Version: 1) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark
partition => INT32
error_code => INT16
high_watermark => INT64
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition_header</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>high_watermark</td><td>Last committed offset.</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Response (Version: 2) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark
partition => INT32
error_code => INT16
high_watermark => INT64
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition_header</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>high_watermark</td><td>Last committed offset.</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Response (Version: 3) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark
partition => INT32
error_code => INT16
high_watermark => INT64
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition_header</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>high_watermark</td><td>Last committed offset.</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Response (Version: 4) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark last_stable_offset [aborted_transactions]
partition => INT32
error_code => INT16
high_watermark => INT64
last_stable_offset => INT64
aborted_transactions => producer_id first_offset
producer_id => INT64
first_offset => INT64
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition_header</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>high_watermark</td><td>Last committed offset.</td></tr>
<tr>
<td>last_stable_offset</td><td>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)</td></tr>
<tr>
<td>aborted_transactions</td><td>null</td></tr>
<tr>
<td>producer_id</td><td>The producer id associated with the aborted transactions</td></tr>
<tr>
<td>first_offset</td><td>The first offset in the aborted transaction</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Response (Version: 5) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions]
partition => INT32
error_code => INT16
high_watermark => INT64
last_stable_offset => INT64
log_start_offset => INT64
aborted_transactions => producer_id first_offset
producer_id => INT64
first_offset => INT64
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition_header</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>high_watermark</td><td>Last committed offset.</td></tr>
<tr>
<td>last_stable_offset</td><td>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)</td></tr>
<tr>
<td>log_start_offset</td><td>Earliest available offset.</td></tr>
<tr>
<td>aborted_transactions</td><td>null</td></tr>
<tr>
<td>producer_id</td><td>The producer id associated with the aborted transactions</td></tr>
<tr>
<td>first_offset</td><td>The first offset in the aborted transaction</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Response (Version: 6) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions]
partition => INT32
error_code => INT16
high_watermark => INT64
last_stable_offset => INT64
log_start_offset => INT64
aborted_transactions => producer_id first_offset
producer_id => INT64
first_offset => INT64
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition_header</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>high_watermark</td><td>Last committed offset.</td></tr>
<tr>
<td>last_stable_offset</td><td>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)</td></tr>
<tr>
<td>log_start_offset</td><td>Earliest available offset.</td></tr>
<tr>
<td>aborted_transactions</td><td>null</td></tr>
<tr>
<td>producer_id</td><td>The producer id associated with the aborted transactions</td></tr>
<tr>
<td>first_offset</td><td>The first offset in the aborted transaction</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Response (Version: 7) => throttle_time_ms error_code session_id [responses]
throttle_time_ms => INT32
error_code => INT16
session_id => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions]
partition => INT32
error_code => INT16
high_watermark => INT64
last_stable_offset => INT64
log_start_offset => INT64
aborted_transactions => producer_id first_offset
producer_id => INT64
first_offset => INT64
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>session_id</td><td>The fetch session ID</td></tr>
<tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition_header</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>high_watermark</td><td>Last committed offset.</td></tr>
<tr>
<td>last_stable_offset</td><td>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)</td></tr>
<tr>
<td>log_start_offset</td><td>Earliest available offset.</td></tr>
<tr>
<td>aborted_transactions</td><td>null</td></tr>
<tr>
<td>producer_id</td><td>The producer id associated with the aborted transactions</td></tr>
<tr>
<td>first_offset</td><td>The first offset in the aborted transaction</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Response (Version: 8) => throttle_time_ms error_code session_id [responses]
throttle_time_ms => INT32
error_code => INT16
session_id => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions]
partition => INT32
error_code => INT16
high_watermark => INT64
last_stable_offset => INT64
log_start_offset => INT64
aborted_transactions => producer_id first_offset
producer_id => INT64
first_offset => INT64
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>session_id</td><td>The fetch session ID</td></tr>
<tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition_header</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>high_watermark</td><td>Last committed offset.</td></tr>
<tr>
<td>last_stable_offset</td><td>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)</td></tr>
<tr>
<td>log_start_offset</td><td>Earliest available offset.</td></tr>
<tr>
<td>aborted_transactions</td><td>null</td></tr>
<tr>
<td>producer_id</td><td>The producer id associated with the aborted transactions</td></tr>
<tr>
<td>first_offset</td><td>The first offset in the aborted transaction</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Response (Version: 9) => throttle_time_ms error_code session_id [responses]
throttle_time_ms => INT32
error_code => INT16
session_id => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions]
partition => INT32
error_code => INT16
high_watermark => INT64
last_stable_offset => INT64
log_start_offset => INT64
aborted_transactions => producer_id first_offset
producer_id => INT64
first_offset => INT64
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>session_id</td><td>The fetch session ID</td></tr>
<tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition_header</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>high_watermark</td><td>Last committed offset.</td></tr>
<tr>
<td>last_stable_offset</td><td>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)</td></tr>
<tr>
<td>log_start_offset</td><td>Earliest available offset.</td></tr>
<tr>
<td>aborted_transactions</td><td>null</td></tr>
<tr>
<td>producer_id</td><td>The producer id associated with the aborted transactions</td></tr>
<tr>
<td>first_offset</td><td>The first offset in the aborted transaction</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Response (Version: 10) => throttle_time_ms error_code session_id [responses]
throttle_time_ms => INT32
error_code => INT16
session_id => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions]
partition => INT32
error_code => INT16
high_watermark => INT64
last_stable_offset => INT64
log_start_offset => INT64
aborted_transactions => producer_id first_offset
producer_id => INT64
first_offset => INT64
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>session_id</td><td>The fetch session ID</td></tr>
<tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition_header</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>high_watermark</td><td>Last committed offset.</td></tr>
<tr>
<td>last_stable_offset</td><td>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)</td></tr>
<tr>
<td>log_start_offset</td><td>Earliest available offset.</td></tr>
<tr>
<td>aborted_transactions</td><td>null</td></tr>
<tr>
<td>producer_id</td><td>The producer id associated with the aborted transactions</td></tr>
<tr>
<td>first_offset</td><td>The first offset in the aborted transaction</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Fetch Response (Version: 11) => throttle_time_ms error_code session_id [responses]
throttle_time_ms => INT32
error_code => INT16
session_id => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition_header record_set
partition_header => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] preferred_read_replica
partition => INT32
error_code => INT16
high_watermark => INT64
last_stable_offset => INT64
log_start_offset => INT64
aborted_transactions => producer_id first_offset
producer_id => INT64
first_offset => INT64
preferred_read_replica => INT32
record_set => RECORDS
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>session_id</td><td>The fetch session ID</td></tr>
<tr>
<td>responses</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>null</td></tr>
<tr>
<td>partition_header</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>high_watermark</td><td>Last committed offset.</td></tr>
<tr>
<td>last_stable_offset</td><td>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)</td></tr>
<tr>
<td>log_start_offset</td><td>Earliest available offset.</td></tr>
<tr>
<td>aborted_transactions</td><td>null</td></tr>
<tr>
<td>producer_id</td><td>The producer id associated with the aborted transactions</td></tr>
<tr>
<td>first_offset</td><td>The first offset in the aborted transaction</td></tr>
<tr>
<td>preferred_read_replica</td><td>The ID of the replica that the consumer should prefer.</td></tr>
<tr>
<td>record_set</td><td>null</td></tr>
</table>
</p>
<h5><a name="The_Messages_ListOffsets">ListOffsets API (Key: 2):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">ListOffsets Request (Version: 0) => replica_id [topics]
replica_id => INT32
topics => topic [partitions]
topic => STRING
partitions => partition timestamp max_num_offsets
partition => INT32
timestamp => INT64
max_num_offsets => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>topics</td><td>Topics to list offsets.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to list offsets.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>timestamp</td><td>The target timestamp for the partition.</td></tr>
<tr>
<td>max_num_offsets</td><td>Maximum offsets to return.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ListOffsets Request (Version: 1) => replica_id [topics]
replica_id => INT32
topics => topic [partitions]
topic => STRING
partitions => partition timestamp
partition => INT32
timestamp => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>topics</td><td>Topics to list offsets.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to list offsets.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>timestamp</td><td>The target timestamp for the partition.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ListOffsets Request (Version: 2) => replica_id isolation_level [topics]
replica_id => INT32
isolation_level => INT8
topics => topic [partitions]
topic => STRING
partitions => partition timestamp
partition => INT32
timestamp => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>isolation_level</td><td>This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records</td></tr>
<tr>
<td>topics</td><td>Topics to list offsets.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to list offsets.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>timestamp</td><td>The target timestamp for the partition.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ListOffsets Request (Version: 3) => replica_id isolation_level [topics]
replica_id => INT32
isolation_level => INT8
topics => topic [partitions]
topic => STRING
partitions => partition timestamp
partition => INT32
timestamp => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>isolation_level</td><td>This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records</td></tr>
<tr>
<td>topics</td><td>Topics to list offsets.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to list offsets.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>timestamp</td><td>The target timestamp for the partition.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ListOffsets Request (Version: 4) => replica_id isolation_level [topics]
replica_id => INT32
isolation_level => INT8
topics => topic [partitions]
topic => STRING
partitions => partition current_leader_epoch timestamp
partition => INT32
current_leader_epoch => INT32
timestamp => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>isolation_level</td><td>This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records</td></tr>
<tr>
<td>topics</td><td>Topics to list offsets.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to list offsets.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>current_leader_epoch</td><td>The current leader epoch, if provided, is used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned.</td></tr>
<tr>
<td>timestamp</td><td>The target timestamp for the partition.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ListOffsets Request (Version: 5) => replica_id isolation_level [topics]
replica_id => INT32
isolation_level => INT8
topics => topic [partitions]
topic => STRING
partitions => partition current_leader_epoch timestamp
partition => INT32
current_leader_epoch => INT32
timestamp => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>isolation_level</td><td>This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records</td></tr>
<tr>
<td>topics</td><td>Topics to list offsets.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>Partitions to list offsets.</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>current_leader_epoch</td><td>The current leader epoch, if provided, is used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned.</td></tr>
<tr>
<td>timestamp</td><td>The target timestamp for the partition.</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">ListOffsets Response (Version: 0) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code [offsets]
partition => INT32
error_code => INT16
offsets => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>responses</td><td>The listed offsets by topic</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>The listed offsets by partition</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>offsets</td><td>A list of offsets.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ListOffsets Response (Version: 1) => [responses]
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code timestamp offset
partition => INT32
error_code => INT16
timestamp => INT64
offset => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>responses</td><td>The listed offsets by topic</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>The listed offsets by partition</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>timestamp</td><td>The timestamp associated with the returned offset</td></tr>
<tr>
<td>offset</td><td>The offset found</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ListOffsets Response (Version: 2) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code timestamp offset
partition => INT32
error_code => INT16
timestamp => INT64
offset => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>responses</td><td>The listed offsets by topic</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>The listed offsets by partition</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>timestamp</td><td>The timestamp associated with the returned offset</td></tr>
<tr>
<td>offset</td><td>The offset found</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ListOffsets Response (Version: 3) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code timestamp offset
partition => INT32
error_code => INT16
timestamp => INT64
offset => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>responses</td><td>The listed offsets by topic</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>The listed offsets by partition</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>timestamp</td><td>The timestamp associated with the returned offset</td></tr>
<tr>
<td>offset</td><td>The offset found</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ListOffsets Response (Version: 4) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code timestamp offset leader_epoch
partition => INT32
error_code => INT16
timestamp => INT64
offset => INT64
leader_epoch => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>responses</td><td>The listed offsets by topic</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>The listed offsets by partition</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>timestamp</td><td>The timestamp associated with the returned offset</td></tr>
<tr>
<td>offset</td><td>The offset found</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ListOffsets Response (Version: 5) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code timestamp offset leader_epoch
partition => INT32
error_code => INT16
timestamp => INT64
offset => INT64
leader_epoch => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>responses</td><td>The listed offsets by topic</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_responses</td><td>The listed offsets by partition</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>timestamp</td><td>The timestamp associated with the returned offset</td></tr>
<tr>
<td>offset</td><td>The offset found</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch</td></tr>
</table>
</p>
<h5><a name="The_Messages_Metadata">Metadata API (Key: 3):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">Metadata Request (Version: 0) => [topics]
topics => name
name => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The topics to fetch metadata for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Request (Version: 1) => [topics]
topics => name
name => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The topics to fetch metadata for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Request (Version: 2) => [topics]
topics => name
name => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The topics to fetch metadata for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Request (Version: 3) => [topics]
topics => name
name => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The topics to fetch metadata for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Request (Version: 4) => [topics] allow_auto_topic_creation
topics => name
name => STRING
allow_auto_topic_creation => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The topics to fetch metadata for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>allow_auto_topic_creation</td><td>If this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Request (Version: 5) => [topics] allow_auto_topic_creation
topics => name
name => STRING
allow_auto_topic_creation => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The topics to fetch metadata for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>allow_auto_topic_creation</td><td>If this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Request (Version: 6) => [topics] allow_auto_topic_creation
topics => name
name => STRING
allow_auto_topic_creation => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The topics to fetch metadata for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>allow_auto_topic_creation</td><td>If this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Request (Version: 7) => [topics] allow_auto_topic_creation
topics => name
name => STRING
allow_auto_topic_creation => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The topics to fetch metadata for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>allow_auto_topic_creation</td><td>If this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Request (Version: 8) => [topics] allow_auto_topic_creation include_cluster_authorized_operations include_topic_authorized_operations
topics => name
name => STRING
allow_auto_topic_creation => BOOLEAN
include_cluster_authorized_operations => BOOLEAN
include_topic_authorized_operations => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The topics to fetch metadata for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>allow_auto_topic_creation</td><td>If this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.</td></tr>
<tr>
<td>include_cluster_authorized_operations</td><td>Whether to include cluster authorized operations.</td></tr>
<tr>
<td>include_topic_authorized_operations</td><td>Whether to include topic authorized operations.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Request (Version: 9) => [topics] allow_auto_topic_creation include_cluster_authorized_operations include_topic_authorized_operations TAG_BUFFER
topics => name TAG_BUFFER
name => COMPACT_STRING
allow_auto_topic_creation => BOOLEAN
include_cluster_authorized_operations => BOOLEAN
include_topic_authorized_operations => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The topics to fetch metadata for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>allow_auto_topic_creation</td><td>If this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so.</td></tr>
<tr>
<td>include_cluster_authorized_operations</td><td>Whether to include cluster authorized operations.</td></tr>
<tr>
<td>include_topic_authorized_operations</td><td>Whether to include topic authorized operations.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">Metadata Response (Version: 0) => [brokers] [topics]
brokers => node_id host port
node_id => INT32
host => STRING
port => INT32
topics => error_code name [partitions]
error_code => INT16
name => STRING
partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes]
error_code => INT16
partition_index => INT32
leader_id => INT32
replica_nodes => INT32
isr_nodes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>brokers</td><td>Each broker in the response.</td></tr>
<tr>
<td>node_id</td><td>The broker ID.</td></tr>
<tr>
<td>host</td><td>The broker hostname.</td></tr>
<tr>
<td>port</td><td>The broker port.</td></tr>
<tr>
<td>topics</td><td>Each topic in the response.</td></tr>
<tr>
<td>error_code</td><td>The topic error, or 0 if there was no error.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>Each partition in the topic.</td></tr>
<tr>
<td>error_code</td><td>The partition error, or 0 if there was no error.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>leader_id</td><td>The ID of the leader broker.</td></tr>
<tr>
<td>replica_nodes</td><td>The set of all nodes that host this partition.</td></tr>
<tr>
<td>isr_nodes</td><td>The set of nodes that are in sync with the leader for this partition.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Response (Version: 1) => [brokers] controller_id [topics]
brokers => node_id host port rack
node_id => INT32
host => STRING
port => INT32
rack => NULLABLE_STRING
controller_id => INT32
topics => error_code name is_internal [partitions]
error_code => INT16
name => STRING
is_internal => BOOLEAN
partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes]
error_code => INT16
partition_index => INT32
leader_id => INT32
replica_nodes => INT32
isr_nodes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>brokers</td><td>Each broker in the response.</td></tr>
<tr>
<td>node_id</td><td>The broker ID.</td></tr>
<tr>
<td>host</td><td>The broker hostname.</td></tr>
<tr>
<td>port</td><td>The broker port.</td></tr>
<tr>
<td>rack</td><td>The rack of the broker, or null if it has not been assigned to a rack.</td></tr>
<tr>
<td>controller_id</td><td>The ID of the controller broker.</td></tr>
<tr>
<td>topics</td><td>Each topic in the response.</td></tr>
<tr>
<td>error_code</td><td>The topic error, or 0 if there was no error.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>is_internal</td><td>True if the topic is internal.</td></tr>
<tr>
<td>partitions</td><td>Each partition in the topic.</td></tr>
<tr>
<td>error_code</td><td>The partition error, or 0 if there was no error.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>leader_id</td><td>The ID of the leader broker.</td></tr>
<tr>
<td>replica_nodes</td><td>The set of all nodes that host this partition.</td></tr>
<tr>
<td>isr_nodes</td><td>The set of nodes that are in sync with the leader for this partition.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Response (Version: 2) => [brokers] cluster_id controller_id [topics]
brokers => node_id host port rack
node_id => INT32
host => STRING
port => INT32
rack => NULLABLE_STRING
cluster_id => NULLABLE_STRING
controller_id => INT32
topics => error_code name is_internal [partitions]
error_code => INT16
name => STRING
is_internal => BOOLEAN
partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes]
error_code => INT16
partition_index => INT32
leader_id => INT32
replica_nodes => INT32
isr_nodes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>brokers</td><td>Each broker in the response.</td></tr>
<tr>
<td>node_id</td><td>The broker ID.</td></tr>
<tr>
<td>host</td><td>The broker hostname.</td></tr>
<tr>
<td>port</td><td>The broker port.</td></tr>
<tr>
<td>rack</td><td>The rack of the broker, or null if it has not been assigned to a rack.</td></tr>
<tr>
<td>cluster_id</td><td>The cluster ID that responding broker belongs to.</td></tr>
<tr>
<td>controller_id</td><td>The ID of the controller broker.</td></tr>
<tr>
<td>topics</td><td>Each topic in the response.</td></tr>
<tr>
<td>error_code</td><td>The topic error, or 0 if there was no error.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>is_internal</td><td>True if the topic is internal.</td></tr>
<tr>
<td>partitions</td><td>Each partition in the topic.</td></tr>
<tr>
<td>error_code</td><td>The partition error, or 0 if there was no error.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>leader_id</td><td>The ID of the leader broker.</td></tr>
<tr>
<td>replica_nodes</td><td>The set of all nodes that host this partition.</td></tr>
<tr>
<td>isr_nodes</td><td>The set of nodes that are in sync with the leader for this partition.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Response (Version: 3) => throttle_time_ms [brokers] cluster_id controller_id [topics]
throttle_time_ms => INT32
brokers => node_id host port rack
node_id => INT32
host => STRING
port => INT32
rack => NULLABLE_STRING
cluster_id => NULLABLE_STRING
controller_id => INT32
topics => error_code name is_internal [partitions]
error_code => INT16
name => STRING
is_internal => BOOLEAN
partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes]
error_code => INT16
partition_index => INT32
leader_id => INT32
replica_nodes => INT32
isr_nodes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>brokers</td><td>Each broker in the response.</td></tr>
<tr>
<td>node_id</td><td>The broker ID.</td></tr>
<tr>
<td>host</td><td>The broker hostname.</td></tr>
<tr>
<td>port</td><td>The broker port.</td></tr>
<tr>
<td>rack</td><td>The rack of the broker, or null if it has not been assigned to a rack.</td></tr>
<tr>
<td>cluster_id</td><td>The cluster ID that responding broker belongs to.</td></tr>
<tr>
<td>controller_id</td><td>The ID of the controller broker.</td></tr>
<tr>
<td>topics</td><td>Each topic in the response.</td></tr>
<tr>
<td>error_code</td><td>The topic error, or 0 if there was no error.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>is_internal</td><td>True if the topic is internal.</td></tr>
<tr>
<td>partitions</td><td>Each partition in the topic.</td></tr>
<tr>
<td>error_code</td><td>The partition error, or 0 if there was no error.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>leader_id</td><td>The ID of the leader broker.</td></tr>
<tr>
<td>replica_nodes</td><td>The set of all nodes that host this partition.</td></tr>
<tr>
<td>isr_nodes</td><td>The set of nodes that are in sync with the leader for this partition.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Response (Version: 4) => throttle_time_ms [brokers] cluster_id controller_id [topics]
throttle_time_ms => INT32
brokers => node_id host port rack
node_id => INT32
host => STRING
port => INT32
rack => NULLABLE_STRING
cluster_id => NULLABLE_STRING
controller_id => INT32
topics => error_code name is_internal [partitions]
error_code => INT16
name => STRING
is_internal => BOOLEAN
partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes]
error_code => INT16
partition_index => INT32
leader_id => INT32
replica_nodes => INT32
isr_nodes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>brokers</td><td>Each broker in the response.</td></tr>
<tr>
<td>node_id</td><td>The broker ID.</td></tr>
<tr>
<td>host</td><td>The broker hostname.</td></tr>
<tr>
<td>port</td><td>The broker port.</td></tr>
<tr>
<td>rack</td><td>The rack of the broker, or null if it has not been assigned to a rack.</td></tr>
<tr>
<td>cluster_id</td><td>The cluster ID that responding broker belongs to.</td></tr>
<tr>
<td>controller_id</td><td>The ID of the controller broker.</td></tr>
<tr>
<td>topics</td><td>Each topic in the response.</td></tr>
<tr>
<td>error_code</td><td>The topic error, or 0 if there was no error.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>is_internal</td><td>True if the topic is internal.</td></tr>
<tr>
<td>partitions</td><td>Each partition in the topic.</td></tr>
<tr>
<td>error_code</td><td>The partition error, or 0 if there was no error.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>leader_id</td><td>The ID of the leader broker.</td></tr>
<tr>
<td>replica_nodes</td><td>The set of all nodes that host this partition.</td></tr>
<tr>
<td>isr_nodes</td><td>The set of nodes that are in sync with the leader for this partition.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Response (Version: 5) => throttle_time_ms [brokers] cluster_id controller_id [topics]
throttle_time_ms => INT32
brokers => node_id host port rack
node_id => INT32
host => STRING
port => INT32
rack => NULLABLE_STRING
cluster_id => NULLABLE_STRING
controller_id => INT32
topics => error_code name is_internal [partitions]
error_code => INT16
name => STRING
is_internal => BOOLEAN
partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes] [offline_replicas]
error_code => INT16
partition_index => INT32
leader_id => INT32
replica_nodes => INT32
isr_nodes => INT32
offline_replicas => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>brokers</td><td>Each broker in the response.</td></tr>
<tr>
<td>node_id</td><td>The broker ID.</td></tr>
<tr>
<td>host</td><td>The broker hostname.</td></tr>
<tr>
<td>port</td><td>The broker port.</td></tr>
<tr>
<td>rack</td><td>The rack of the broker, or null if it has not been assigned to a rack.</td></tr>
<tr>
<td>cluster_id</td><td>The cluster ID that responding broker belongs to.</td></tr>
<tr>
<td>controller_id</td><td>The ID of the controller broker.</td></tr>
<tr>
<td>topics</td><td>Each topic in the response.</td></tr>
<tr>
<td>error_code</td><td>The topic error, or 0 if there was no error.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>is_internal</td><td>True if the topic is internal.</td></tr>
<tr>
<td>partitions</td><td>Each partition in the topic.</td></tr>
<tr>
<td>error_code</td><td>The partition error, or 0 if there was no error.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>leader_id</td><td>The ID of the leader broker.</td></tr>
<tr>
<td>replica_nodes</td><td>The set of all nodes that host this partition.</td></tr>
<tr>
<td>isr_nodes</td><td>The set of nodes that are in sync with the leader for this partition.</td></tr>
<tr>
<td>offline_replicas</td><td>The set of offline replicas of this partition.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Response (Version: 6) => throttle_time_ms [brokers] cluster_id controller_id [topics]
throttle_time_ms => INT32
brokers => node_id host port rack
node_id => INT32
host => STRING
port => INT32
rack => NULLABLE_STRING
cluster_id => NULLABLE_STRING
controller_id => INT32
topics => error_code name is_internal [partitions]
error_code => INT16
name => STRING
is_internal => BOOLEAN
partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes] [offline_replicas]
error_code => INT16
partition_index => INT32
leader_id => INT32
replica_nodes => INT32
isr_nodes => INT32
offline_replicas => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>brokers</td><td>Each broker in the response.</td></tr>
<tr>
<td>node_id</td><td>The broker ID.</td></tr>
<tr>
<td>host</td><td>The broker hostname.</td></tr>
<tr>
<td>port</td><td>The broker port.</td></tr>
<tr>
<td>rack</td><td>The rack of the broker, or null if it has not been assigned to a rack.</td></tr>
<tr>
<td>cluster_id</td><td>The cluster ID that responding broker belongs to.</td></tr>
<tr>
<td>controller_id</td><td>The ID of the controller broker.</td></tr>
<tr>
<td>topics</td><td>Each topic in the response.</td></tr>
<tr>
<td>error_code</td><td>The topic error, or 0 if there was no error.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>is_internal</td><td>True if the topic is internal.</td></tr>
<tr>
<td>partitions</td><td>Each partition in the topic.</td></tr>
<tr>
<td>error_code</td><td>The partition error, or 0 if there was no error.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>leader_id</td><td>The ID of the leader broker.</td></tr>
<tr>
<td>replica_nodes</td><td>The set of all nodes that host this partition.</td></tr>
<tr>
<td>isr_nodes</td><td>The set of nodes that are in sync with the leader for this partition.</td></tr>
<tr>
<td>offline_replicas</td><td>The set of offline replicas of this partition.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Response (Version: 7) => throttle_time_ms [brokers] cluster_id controller_id [topics]
throttle_time_ms => INT32
brokers => node_id host port rack
node_id => INT32
host => STRING
port => INT32
rack => NULLABLE_STRING
cluster_id => NULLABLE_STRING
controller_id => INT32
topics => error_code name is_internal [partitions]
error_code => INT16
name => STRING
is_internal => BOOLEAN
partitions => error_code partition_index leader_id leader_epoch [replica_nodes] [isr_nodes] [offline_replicas]
error_code => INT16
partition_index => INT32
leader_id => INT32
leader_epoch => INT32
replica_nodes => INT32
isr_nodes => INT32
offline_replicas => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>brokers</td><td>Each broker in the response.</td></tr>
<tr>
<td>node_id</td><td>The broker ID.</td></tr>
<tr>
<td>host</td><td>The broker hostname.</td></tr>
<tr>
<td>port</td><td>The broker port.</td></tr>
<tr>
<td>rack</td><td>The rack of the broker, or null if it has not been assigned to a rack.</td></tr>
<tr>
<td>cluster_id</td><td>The cluster ID that responding broker belongs to.</td></tr>
<tr>
<td>controller_id</td><td>The ID of the controller broker.</td></tr>
<tr>
<td>topics</td><td>Each topic in the response.</td></tr>
<tr>
<td>error_code</td><td>The topic error, or 0 if there was no error.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>is_internal</td><td>True if the topic is internal.</td></tr>
<tr>
<td>partitions</td><td>Each partition in the topic.</td></tr>
<tr>
<td>error_code</td><td>The partition error, or 0 if there was no error.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>leader_id</td><td>The ID of the leader broker.</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch of this partition.</td></tr>
<tr>
<td>replica_nodes</td><td>The set of all nodes that host this partition.</td></tr>
<tr>
<td>isr_nodes</td><td>The set of nodes that are in sync with the leader for this partition.</td></tr>
<tr>
<td>offline_replicas</td><td>The set of offline replicas of this partition.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Response (Version: 8) => throttle_time_ms [brokers] cluster_id controller_id [topics] cluster_authorized_operations
throttle_time_ms => INT32
brokers => node_id host port rack
node_id => INT32
host => STRING
port => INT32
rack => NULLABLE_STRING
cluster_id => NULLABLE_STRING
controller_id => INT32
topics => error_code name is_internal [partitions] topic_authorized_operations
error_code => INT16
name => STRING
is_internal => BOOLEAN
partitions => error_code partition_index leader_id leader_epoch [replica_nodes] [isr_nodes] [offline_replicas]
error_code => INT16
partition_index => INT32
leader_id => INT32
leader_epoch => INT32
replica_nodes => INT32
isr_nodes => INT32
offline_replicas => INT32
topic_authorized_operations => INT32
cluster_authorized_operations => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>brokers</td><td>Each broker in the response.</td></tr>
<tr>
<td>node_id</td><td>The broker ID.</td></tr>
<tr>
<td>host</td><td>The broker hostname.</td></tr>
<tr>
<td>port</td><td>The broker port.</td></tr>
<tr>
<td>rack</td><td>The rack of the broker, or null if it has not been assigned to a rack.</td></tr>
<tr>
<td>cluster_id</td><td>The cluster ID that responding broker belongs to.</td></tr>
<tr>
<td>controller_id</td><td>The ID of the controller broker.</td></tr>
<tr>
<td>topics</td><td>Each topic in the response.</td></tr>
<tr>
<td>error_code</td><td>The topic error, or 0 if there was no error.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>is_internal</td><td>True if the topic is internal.</td></tr>
<tr>
<td>partitions</td><td>Each partition in the topic.</td></tr>
<tr>
<td>error_code</td><td>The partition error, or 0 if there was no error.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>leader_id</td><td>The ID of the leader broker.</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch of this partition.</td></tr>
<tr>
<td>replica_nodes</td><td>The set of all nodes that host this partition.</td></tr>
<tr>
<td>isr_nodes</td><td>The set of nodes that are in sync with the leader for this partition.</td></tr>
<tr>
<td>offline_replicas</td><td>The set of offline replicas of this partition.</td></tr>
<tr>
<td>topic_authorized_operations</td><td>32-bit bitfield to represent authorized operations for this topic.</td></tr>
<tr>
<td>cluster_authorized_operations</td><td>32-bit bitfield to represent authorized operations for this cluster.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Metadata Response (Version: 9) => throttle_time_ms [brokers] cluster_id controller_id [topics] cluster_authorized_operations TAG_BUFFER
throttle_time_ms => INT32
brokers => node_id host port rack TAG_BUFFER
node_id => INT32
host => COMPACT_STRING
port => INT32
rack => COMPACT_NULLABLE_STRING
cluster_id => COMPACT_NULLABLE_STRING
controller_id => INT32
topics => error_code name is_internal [partitions] topic_authorized_operations TAG_BUFFER
error_code => INT16
name => COMPACT_STRING
is_internal => BOOLEAN
partitions => error_code partition_index leader_id leader_epoch [replica_nodes] [isr_nodes] [offline_replicas] TAG_BUFFER
error_code => INT16
partition_index => INT32
leader_id => INT32
leader_epoch => INT32
replica_nodes => INT32
isr_nodes => INT32
offline_replicas => INT32
topic_authorized_operations => INT32
cluster_authorized_operations => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>brokers</td><td>Each broker in the response.</td></tr>
<tr>
<td>node_id</td><td>The broker ID.</td></tr>
<tr>
<td>host</td><td>The broker hostname.</td></tr>
<tr>
<td>port</td><td>The broker port.</td></tr>
<tr>
<td>rack</td><td>The rack of the broker, or null if it has not been assigned to a rack.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>cluster_id</td><td>The cluster ID that responding broker belongs to.</td></tr>
<tr>
<td>controller_id</td><td>The ID of the controller broker.</td></tr>
<tr>
<td>topics</td><td>Each topic in the response.</td></tr>
<tr>
<td>error_code</td><td>The topic error, or 0 if there was no error.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>is_internal</td><td>True if the topic is internal.</td></tr>
<tr>
<td>partitions</td><td>Each partition in the topic.</td></tr>
<tr>
<td>error_code</td><td>The partition error, or 0 if there was no error.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>leader_id</td><td>The ID of the leader broker.</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch of this partition.</td></tr>
<tr>
<td>replica_nodes</td><td>The set of all nodes that host this partition.</td></tr>
<tr>
<td>isr_nodes</td><td>The set of nodes that are in sync with the leader for this partition.</td></tr>
<tr>
<td>offline_replicas</td><td>The set of offline replicas of this partition.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>topic_authorized_operations</td><td>32-bit bitfield to represent authorized operations for this topic.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>cluster_authorized_operations</td><td>32-bit bitfield to represent authorized operations for this cluster.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_LeaderAndIsr">LeaderAndIsr API (Key: 4):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">LeaderAndIsr Request (Version: 0) => controller_id controller_epoch [ungrouped_partition_states] [live_leaders]
controller_id => INT32
controller_epoch => INT32
ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas]
topic_name => STRING
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
live_leaders => broker_id host_name port
broker_id => INT32
host_name => STRING
port => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>controller_id</td><td>The current controller ID.</td></tr>
<tr>
<td>controller_epoch</td><td>The current controller epoch.</td></tr>
<tr>
<td>ungrouped_partition_states</td><td>The state of each partition, in a v0 or v1 message.</td></tr>
<tr>
<td>topic_name</td><td>The topic name. This is only present in v0 or v1.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>leader</td><td>The broker ID of the leader.</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch.</td></tr>
<tr>
<td>isr</td><td>The in-sync replica IDs.</td></tr>
<tr>
<td>zk_version</td><td>The ZooKeeper version.</td></tr>
<tr>
<td>replicas</td><td>The replica IDs.</td></tr>
<tr>
<td>live_leaders</td><td>The current live leaders.</td></tr>
<tr>
<td>broker_id</td><td>The leader's broker ID.</td></tr>
<tr>
<td>host_name</td><td>The leader's hostname.</td></tr>
<tr>
<td>port</td><td>The leader's port.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">LeaderAndIsr Request (Version: 1) => controller_id controller_epoch [ungrouped_partition_states] [live_leaders]
controller_id => INT32
controller_epoch => INT32
ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] is_new
topic_name => STRING
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
is_new => BOOLEAN
live_leaders => broker_id host_name port
broker_id => INT32
host_name => STRING
port => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>controller_id</td><td>The current controller ID.</td></tr>
<tr>
<td>controller_epoch</td><td>The current controller epoch.</td></tr>
<tr>
<td>ungrouped_partition_states</td><td>The state of each partition, in a v0 or v1 message.</td></tr>
<tr>
<td>topic_name</td><td>The topic name. This is only present in v0 or v1.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>leader</td><td>The broker ID of the leader.</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch.</td></tr>
<tr>
<td>isr</td><td>The in-sync replica IDs.</td></tr>
<tr>
<td>zk_version</td><td>The ZooKeeper version.</td></tr>
<tr>
<td>replicas</td><td>The replica IDs.</td></tr>
<tr>
<td>is_new</td><td>Whether the replica should have existed on the broker or not.</td></tr>
<tr>
<td>live_leaders</td><td>The current live leaders.</td></tr>
<tr>
<td>broker_id</td><td>The leader's broker ID.</td></tr>
<tr>
<td>host_name</td><td>The leader's hostname.</td></tr>
<tr>
<td>port</td><td>The leader's port.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">LeaderAndIsr Request (Version: 2) => controller_id controller_epoch broker_epoch [topic_states] [live_leaders]
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
topic_states => topic_name [partition_states]
topic_name => STRING
partition_states => partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] is_new
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
is_new => BOOLEAN
live_leaders => broker_id host_name port
broker_id => INT32
host_name => STRING
port => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>controller_id</td><td>The current controller ID.</td></tr>
<tr>
<td>controller_epoch</td><td>The current controller epoch.</td></tr>
<tr>
<td>broker_epoch</td><td>The current broker epoch.</td></tr>
<tr>
<td>topic_states</td><td>Each topic.</td></tr>
<tr>
<td>topic_name</td><td>The topic name.</td></tr>
<tr>
<td>partition_states</td><td>The state of each partition</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>leader</td><td>The broker ID of the leader.</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch.</td></tr>
<tr>
<td>isr</td><td>The in-sync replica IDs.</td></tr>
<tr>
<td>zk_version</td><td>The ZooKeeper version.</td></tr>
<tr>
<td>replicas</td><td>The replica IDs.</td></tr>
<tr>
<td>is_new</td><td>Whether the replica should have existed on the broker or not.</td></tr>
<tr>
<td>live_leaders</td><td>The current live leaders.</td></tr>
<tr>
<td>broker_id</td><td>The leader's broker ID.</td></tr>
<tr>
<td>host_name</td><td>The leader's hostname.</td></tr>
<tr>
<td>port</td><td>The leader's port.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">LeaderAndIsr Request (Version: 3) => controller_id controller_epoch broker_epoch [topic_states] [live_leaders]
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
topic_states => topic_name [partition_states]
topic_name => STRING
partition_states => partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] [adding_replicas] [removing_replicas] is_new
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
adding_replicas => INT32
removing_replicas => INT32
is_new => BOOLEAN
live_leaders => broker_id host_name port
broker_id => INT32
host_name => STRING
port => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>controller_id</td><td>The current controller ID.</td></tr>
<tr>
<td>controller_epoch</td><td>The current controller epoch.</td></tr>
<tr>
<td>broker_epoch</td><td>The current broker epoch.</td></tr>
<tr>
<td>topic_states</td><td>Each topic.</td></tr>
<tr>
<td>topic_name</td><td>The topic name.</td></tr>
<tr>
<td>partition_states</td><td>The state of each partition</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>leader</td><td>The broker ID of the leader.</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch.</td></tr>
<tr>
<td>isr</td><td>The in-sync replica IDs.</td></tr>
<tr>
<td>zk_version</td><td>The ZooKeeper version.</td></tr>
<tr>
<td>replicas</td><td>The replica IDs.</td></tr>
<tr>
<td>adding_replicas</td><td>The replica IDs that we are adding this partition to, or null if no replicas are being added.</td></tr>
<tr>
<td>removing_replicas</td><td>The replica IDs that we are removing this partition from, or null if no replicas are being removed.</td></tr>
<tr>
<td>is_new</td><td>Whether the replica should have existed on the broker or not.</td></tr>
<tr>
<td>live_leaders</td><td>The current live leaders.</td></tr>
<tr>
<td>broker_id</td><td>The leader's broker ID.</td></tr>
<tr>
<td>host_name</td><td>The leader's hostname.</td></tr>
<tr>
<td>port</td><td>The leader's port.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">LeaderAndIsr Request (Version: 4) => controller_id controller_epoch broker_epoch [topic_states] [live_leaders] TAG_BUFFER
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
topic_states => topic_name [partition_states] TAG_BUFFER
topic_name => COMPACT_STRING
partition_states => partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] [adding_replicas] [removing_replicas] is_new TAG_BUFFER
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
adding_replicas => INT32
removing_replicas => INT32
is_new => BOOLEAN
live_leaders => broker_id host_name port TAG_BUFFER
broker_id => INT32
host_name => COMPACT_STRING
port => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>controller_id</td><td>The current controller ID.</td></tr>
<tr>
<td>controller_epoch</td><td>The current controller epoch.</td></tr>
<tr>
<td>broker_epoch</td><td>The current broker epoch.</td></tr>
<tr>
<td>topic_states</td><td>Each topic.</td></tr>
<tr>
<td>topic_name</td><td>The topic name.</td></tr>
<tr>
<td>partition_states</td><td>The state of each partition</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>leader</td><td>The broker ID of the leader.</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch.</td></tr>
<tr>
<td>isr</td><td>The in-sync replica IDs.</td></tr>
<tr>
<td>zk_version</td><td>The ZooKeeper version.</td></tr>
<tr>
<td>replicas</td><td>The replica IDs.</td></tr>
<tr>
<td>adding_replicas</td><td>The replica IDs that we are adding this partition to, or null if no replicas are being added.</td></tr>
<tr>
<td>removing_replicas</td><td>The replica IDs that we are removing this partition from, or null if no replicas are being removed.</td></tr>
<tr>
<td>is_new</td><td>Whether the replica should have existed on the broker or not.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>live_leaders</td><td>The current live leaders.</td></tr>
<tr>
<td>broker_id</td><td>The leader's broker ID.</td></tr>
<tr>
<td>host_name</td><td>The leader's hostname.</td></tr>
<tr>
<td>port</td><td>The leader's port.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">LeaderAndIsr Response (Version: 0) => error_code [partition_errors]
error_code => INT16
partition_errors => topic_name partition_index error_code
topic_name => STRING
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>partition_errors</td><td>Each partition.</td></tr>
<tr>
<td>topic_name</td><td>The topic name.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The partition error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">LeaderAndIsr Response (Version: 1) => error_code [partition_errors]
error_code => INT16
partition_errors => topic_name partition_index error_code
topic_name => STRING
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>partition_errors</td><td>Each partition.</td></tr>
<tr>
<td>topic_name</td><td>The topic name.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The partition error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">LeaderAndIsr Response (Version: 2) => error_code [partition_errors]
error_code => INT16
partition_errors => topic_name partition_index error_code
topic_name => STRING
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>partition_errors</td><td>Each partition.</td></tr>
<tr>
<td>topic_name</td><td>The topic name.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The partition error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">LeaderAndIsr Response (Version: 3) => error_code [partition_errors]
error_code => INT16
partition_errors => topic_name partition_index error_code
topic_name => STRING
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>partition_errors</td><td>Each partition.</td></tr>
<tr>
<td>topic_name</td><td>The topic name.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The partition error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">LeaderAndIsr Response (Version: 4) => error_code [partition_errors] TAG_BUFFER
error_code => INT16
partition_errors => topic_name partition_index error_code TAG_BUFFER
topic_name => COMPACT_STRING
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>partition_errors</td><td>Each partition.</td></tr>
<tr>
<td>topic_name</td><td>The topic name.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The partition error code, or 0 if there was no error.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_StopReplica">StopReplica API (Key: 5):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">StopReplica Request (Version: 0) => controller_id controller_epoch delete_partitions [ungrouped_partitions]
controller_id => INT32
controller_epoch => INT32
delete_partitions => BOOLEAN
ungrouped_partitions => topic_name partition_index
topic_name => STRING
partition_index => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>controller_id</td><td>The controller id.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>delete_partitions</td><td>Whether these partitions should be deleted.</td></tr>
<tr>
<td>ungrouped_partitions</td><td>The partitions to stop.</td></tr>
<tr>
<td>topic_name</td><td>The topic name.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">StopReplica Request (Version: 1) => controller_id controller_epoch broker_epoch delete_partitions [topics]
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
delete_partitions => BOOLEAN
topics => name [partition_indexes]
name => STRING
partition_indexes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>controller_id</td><td>The controller id.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>broker_epoch</td><td>The broker epoch.</td></tr>
<tr>
<td>delete_partitions</td><td>Whether these partitions should be deleted.</td></tr>
<tr>
<td>topics</td><td>The topics to stop.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partition_indexes</td><td>The partition indexes.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">StopReplica Request (Version: 2) => controller_id controller_epoch broker_epoch delete_partitions [topics] TAG_BUFFER
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
delete_partitions => BOOLEAN
topics => name [partition_indexes] TAG_BUFFER
name => COMPACT_STRING
partition_indexes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>controller_id</td><td>The controller id.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>broker_epoch</td><td>The broker epoch.</td></tr>
<tr>
<td>delete_partitions</td><td>Whether these partitions should be deleted.</td></tr>
<tr>
<td>topics</td><td>The topics to stop.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partition_indexes</td><td>The partition indexes.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">StopReplica Response (Version: 0) => error_code [partition_errors]
error_code => INT16
partition_errors => topic_name partition_index error_code
topic_name => STRING
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The top-level error code, or 0 if there was no top-level error.</td></tr>
<tr>
<td>partition_errors</td><td>The responses for each partition.</td></tr>
<tr>
<td>topic_name</td><td>The topic name.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The partition error code, or 0 if there was no partition error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">StopReplica Response (Version: 1) => error_code [partition_errors]
error_code => INT16
partition_errors => topic_name partition_index error_code
topic_name => STRING
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The top-level error code, or 0 if there was no top-level error.</td></tr>
<tr>
<td>partition_errors</td><td>The responses for each partition.</td></tr>
<tr>
<td>topic_name</td><td>The topic name.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The partition error code, or 0 if there was no partition error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">StopReplica Response (Version: 2) => error_code [partition_errors] TAG_BUFFER
error_code => INT16
partition_errors => topic_name partition_index error_code TAG_BUFFER
topic_name => COMPACT_STRING
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The top-level error code, or 0 if there was no top-level error.</td></tr>
<tr>
<td>partition_errors</td><td>The responses for each partition.</td></tr>
<tr>
<td>topic_name</td><td>The topic name.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The partition error code, or 0 if there was no partition error.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_UpdateMetadata">UpdateMetadata API (Key: 6):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">UpdateMetadata Request (Version: 0) => controller_id controller_epoch [ungrouped_partition_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas]
topic_name => STRING
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
live_brokers => id v0_host v0_port
id => INT32
v0_host => STRING
v0_port => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>controller_id</td><td>The controller id.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>ungrouped_partition_states</td><td>In older versions of this RPC, each partition that we would like to update.</td></tr>
<tr>
<td>topic_name</td><td>In older versions of this RPC, the topic name.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>leader</td><td>The ID of the broker which is the current partition leader.</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch of this partition.</td></tr>
<tr>
<td>isr</td><td>The brokers which are in the ISR for this partition.</td></tr>
<tr>
<td>zk_version</td><td>The Zookeeper version.</td></tr>
<tr>
<td>replicas</td><td>All the replicas of this partition.</td></tr>
<tr>
<td>live_brokers</td><td></td></tr>
<tr>
<td>id</td><td>The broker id.</td></tr>
<tr>
<td>v0_host</td><td>The broker hostname.</td></tr>
<tr>
<td>v0_port</td><td>The broker port.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">UpdateMetadata Request (Version: 1) => controller_id controller_epoch [ungrouped_partition_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas]
topic_name => STRING
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
live_brokers => id [endpoints]
id => INT32
endpoints => port host security_protocol
port => INT32
host => STRING
security_protocol => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>controller_id</td><td>The controller id.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>ungrouped_partition_states</td><td>In older versions of this RPC, each partition that we would like to update.</td></tr>
<tr>
<td>topic_name</td><td>In older versions of this RPC, the topic name.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>leader</td><td>The ID of the broker which is the current partition leader.</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch of this partition.</td></tr>
<tr>
<td>isr</td><td>The brokers which are in the ISR for this partition.</td></tr>
<tr>
<td>zk_version</td><td>The Zookeeper version.</td></tr>
<tr>
<td>replicas</td><td>All the replicas of this partition.</td></tr>
<tr>
<td>live_brokers</td><td></td></tr>
<tr>
<td>id</td><td>The broker id.</td></tr>
<tr>
<td>endpoints</td><td>The broker endpoints.</td></tr>
<tr>
<td>port</td><td>The port of this endpoint</td></tr>
<tr>
<td>host</td><td>The hostname of this endpoint</td></tr>
<tr>
<td>security_protocol</td><td>The security protocol type.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">UpdateMetadata Request (Version: 2) => controller_id controller_epoch [ungrouped_partition_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas]
topic_name => STRING
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
live_brokers => id [endpoints] rack
id => INT32
endpoints => port host security_protocol
port => INT32
host => STRING
security_protocol => INT16
rack => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>controller_id</td><td>The controller id.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>ungrouped_partition_states</td><td>In older versions of this RPC, each partition that we would like to update.</td></tr>
<tr>
<td>topic_name</td><td>In older versions of this RPC, the topic name.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>leader</td><td>The ID of the broker which is the current partition leader.</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch of this partition.</td></tr>
<tr>
<td>isr</td><td>The brokers which are in the ISR for this partition.</td></tr>
<tr>
<td>zk_version</td><td>The Zookeeper version.</td></tr>
<tr>
<td>replicas</td><td>All the replicas of this partition.</td></tr>
<tr>
<td>live_brokers</td><td></td></tr>
<tr>
<td>id</td><td>The broker id.</td></tr>
<tr>
<td>endpoints</td><td>The broker endpoints.</td></tr>
<tr>
<td>port</td><td>The port of this endpoint</td></tr>
<tr>
<td>host</td><td>The hostname of this endpoint</td></tr>
<tr>
<td>security_protocol</td><td>The security protocol type.</td></tr>
<tr>
<td>rack</td><td>The rack which this broker belongs to.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">UpdateMetadata Request (Version: 3) => controller_id controller_epoch [ungrouped_partition_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas]
topic_name => STRING
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
live_brokers => id [endpoints] rack
id => INT32
endpoints => port host listener security_protocol
port => INT32
host => STRING
listener => STRING
security_protocol => INT16
rack => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>controller_id</td><td>The controller id.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>ungrouped_partition_states</td><td>In older versions of this RPC, each partition that we would like to update.</td></tr>
<tr>
<td>topic_name</td><td>In older versions of this RPC, the topic name.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>leader</td><td>The ID of the broker which is the current partition leader.</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch of this partition.</td></tr>
<tr>
<td>isr</td><td>The brokers which are in the ISR for this partition.</td></tr>
<tr>
<td>zk_version</td><td>The Zookeeper version.</td></tr>
<tr>
<td>replicas</td><td>All the replicas of this partition.</td></tr>
<tr>
<td>live_brokers</td><td></td></tr>
<tr>
<td>id</td><td>The broker id.</td></tr>
<tr>
<td>endpoints</td><td>The broker endpoints.</td></tr>
<tr>
<td>port</td><td>The port of this endpoint</td></tr>
<tr>
<td>host</td><td>The hostname of this endpoint</td></tr>
<tr>
<td>listener</td><td>The listener name.</td></tr>
<tr>
<td>security_protocol</td><td>The security protocol type.</td></tr>
<tr>
<td>rack</td><td>The rack which this broker belongs to.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">UpdateMetadata Request (Version: 4) => controller_id controller_epoch [ungrouped_partition_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] [offline_replicas]
topic_name => STRING
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
offline_replicas => INT32
live_brokers => id [endpoints] rack
id => INT32
endpoints => port host listener security_protocol
port => INT32
host => STRING
listener => STRING
security_protocol => INT16
rack => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>controller_id</td><td>The controller id.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>ungrouped_partition_states</td><td>In older versions of this RPC, each partition that we would like to update.</td></tr>
<tr>
<td>topic_name</td><td>In older versions of this RPC, the topic name.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>leader</td><td>The ID of the broker which is the current partition leader.</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch of this partition.</td></tr>
<tr>
<td>isr</td><td>The brokers which are in the ISR for this partition.</td></tr>
<tr>
<td>zk_version</td><td>The Zookeeper version.</td></tr>
<tr>
<td>replicas</td><td>All the replicas of this partition.</td></tr>
<tr>
<td>offline_replicas</td><td>The replicas of this partition which are offline.</td></tr>
<tr>
<td>live_brokers</td><td></td></tr>
<tr>
<td>id</td><td>The broker id.</td></tr>
<tr>
<td>endpoints</td><td>The broker endpoints.</td></tr>
<tr>
<td>port</td><td>The port of this endpoint</td></tr>
<tr>
<td>host</td><td>The hostname of this endpoint</td></tr>
<tr>
<td>listener</td><td>The listener name.</td></tr>
<tr>
<td>security_protocol</td><td>The security protocol type.</td></tr>
<tr>
<td>rack</td><td>The rack which this broker belongs to.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">UpdateMetadata Request (Version: 5) => controller_id controller_epoch broker_epoch [topic_states] [live_brokers]
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
topic_states => topic_name [partition_states]
topic_name => STRING
partition_states => partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] [offline_replicas]
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
offline_replicas => INT32
live_brokers => id [endpoints] rack
id => INT32
endpoints => port host listener security_protocol
port => INT32
host => STRING
listener => STRING
security_protocol => INT16
rack => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>controller_id</td><td>The controller id.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>broker_epoch</td><td>The broker epoch.</td></tr>
<tr>
<td>topic_states</td><td>In newer versions of this RPC, each topic that we would like to update.</td></tr>
<tr>
<td>topic_name</td><td>The topic name.</td></tr>
<tr>
<td>partition_states</td><td>The partition that we would like to update.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>leader</td><td>The ID of the broker which is the current partition leader.</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch of this partition.</td></tr>
<tr>
<td>isr</td><td>The brokers which are in the ISR for this partition.</td></tr>
<tr>
<td>zk_version</td><td>The Zookeeper version.</td></tr>
<tr>
<td>replicas</td><td>All the replicas of this partition.</td></tr>
<tr>
<td>offline_replicas</td><td>The replicas of this partition which are offline.</td></tr>
<tr>
<td>live_brokers</td><td></td></tr>
<tr>
<td>id</td><td>The broker id.</td></tr>
<tr>
<td>endpoints</td><td>The broker endpoints.</td></tr>
<tr>
<td>port</td><td>The port of this endpoint</td></tr>
<tr>
<td>host</td><td>The hostname of this endpoint</td></tr>
<tr>
<td>listener</td><td>The listener name.</td></tr>
<tr>
<td>security_protocol</td><td>The security protocol type.</td></tr>
<tr>
<td>rack</td><td>The rack which this broker belongs to.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">UpdateMetadata Request (Version: 6) => controller_id controller_epoch broker_epoch [topic_states] [live_brokers] TAG_BUFFER
controller_id => INT32
controller_epoch => INT32
broker_epoch => INT64
topic_states => topic_name [partition_states] TAG_BUFFER
topic_name => COMPACT_STRING
partition_states => partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] [offline_replicas] TAG_BUFFER
partition_index => INT32
controller_epoch => INT32
leader => INT32
leader_epoch => INT32
isr => INT32
zk_version => INT32
replicas => INT32
offline_replicas => INT32
live_brokers => id [endpoints] rack TAG_BUFFER
id => INT32
endpoints => port host listener security_protocol TAG_BUFFER
port => INT32
host => COMPACT_STRING
listener => COMPACT_STRING
security_protocol => INT16
rack => COMPACT_NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>controller_id</td><td>The controller id.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>broker_epoch</td><td>The broker epoch.</td></tr>
<tr>
<td>topic_states</td><td>In newer versions of this RPC, each topic that we would like to update.</td></tr>
<tr>
<td>topic_name</td><td>The topic name.</td></tr>
<tr>
<td>partition_states</td><td>The partition that we would like to update.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>controller_epoch</td><td>The controller epoch.</td></tr>
<tr>
<td>leader</td><td>The ID of the broker which is the current partition leader.</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch of this partition.</td></tr>
<tr>
<td>isr</td><td>The brokers which are in the ISR for this partition.</td></tr>
<tr>
<td>zk_version</td><td>The Zookeeper version.</td></tr>
<tr>
<td>replicas</td><td>All the replicas of this partition.</td></tr>
<tr>
<td>offline_replicas</td><td>The replicas of this partition which are offline.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>live_brokers</td><td></td></tr>
<tr>
<td>id</td><td>The broker id.</td></tr>
<tr>
<td>endpoints</td><td>The broker endpoints.</td></tr>
<tr>
<td>port</td><td>The port of this endpoint</td></tr>
<tr>
<td>host</td><td>The hostname of this endpoint</td></tr>
<tr>
<td>listener</td><td>The listener name.</td></tr>
<tr>
<td>security_protocol</td><td>The security protocol type.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>rack</td><td>The rack which this broker belongs to.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">UpdateMetadata Response (Version: 0) => error_code
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">UpdateMetadata Response (Version: 1) => error_code
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">UpdateMetadata Response (Version: 2) => error_code
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">UpdateMetadata Response (Version: 3) => error_code
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">UpdateMetadata Response (Version: 4) => error_code
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">UpdateMetadata Response (Version: 5) => error_code
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">UpdateMetadata Response (Version: 6) => error_code TAG_BUFFER
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_ControlledShutdown">ControlledShutdown API (Key: 7):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">ControlledShutdown Request (Version: 0) => broker_id
broker_id => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>broker_id</td><td>The id of the broker for which controlled shutdown has been requested.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ControlledShutdown Request (Version: 1) => broker_id
broker_id => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>broker_id</td><td>The id of the broker for which controlled shutdown has been requested.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ControlledShutdown Request (Version: 2) => broker_id broker_epoch
broker_id => INT32
broker_epoch => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>broker_id</td><td>The id of the broker for which controlled shutdown has been requested.</td></tr>
<tr>
<td>broker_epoch</td><td>The broker epoch.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ControlledShutdown Request (Version: 3) => broker_id broker_epoch TAG_BUFFER
broker_id => INT32
broker_epoch => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>broker_id</td><td>The id of the broker for which controlled shutdown has been requested.</td></tr>
<tr>
<td>broker_epoch</td><td>The broker epoch.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">ControlledShutdown Response (Version: 0) => error_code [remaining_partitions]
error_code => INT16
remaining_partitions => topic_name partition_index
topic_name => STRING
partition_index => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The top-level error code.</td></tr>
<tr>
<td>remaining_partitions</td><td>The partitions that the broker still leads.</td></tr>
<tr>
<td>topic_name</td><td>The name of the topic.</td></tr>
<tr>
<td>partition_index</td><td>The index of the partition.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ControlledShutdown Response (Version: 1) => error_code [remaining_partitions]
error_code => INT16
remaining_partitions => topic_name partition_index
topic_name => STRING
partition_index => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The top-level error code.</td></tr>
<tr>
<td>remaining_partitions</td><td>The partitions that the broker still leads.</td></tr>
<tr>
<td>topic_name</td><td>The name of the topic.</td></tr>
<tr>
<td>partition_index</td><td>The index of the partition.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ControlledShutdown Response (Version: 2) => error_code [remaining_partitions]
error_code => INT16
remaining_partitions => topic_name partition_index
topic_name => STRING
partition_index => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The top-level error code.</td></tr>
<tr>
<td>remaining_partitions</td><td>The partitions that the broker still leads.</td></tr>
<tr>
<td>topic_name</td><td>The name of the topic.</td></tr>
<tr>
<td>partition_index</td><td>The index of the partition.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ControlledShutdown Response (Version: 3) => error_code [remaining_partitions] TAG_BUFFER
error_code => INT16
remaining_partitions => topic_name partition_index TAG_BUFFER
topic_name => COMPACT_STRING
partition_index => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The top-level error code.</td></tr>
<tr>
<td>remaining_partitions</td><td>The partitions that the broker still leads.</td></tr>
<tr>
<td>topic_name</td><td>The name of the topic.</td></tr>
<tr>
<td>partition_index</td><td>The index of the partition.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_OffsetCommit">OffsetCommit API (Key: 8):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Request (Version: 0) => group_id [topics]
group_id => STRING
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset committed_metadata
partition_index => INT32
committed_offset => INT64
committed_metadata => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The unique group identifier.</td></tr>
<tr>
<td>topics</td><td>The topics to commit offsets for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>Each partition to commit offsets for.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>committed_offset</td><td>The message offset to be committed.</td></tr>
<tr>
<td>committed_metadata</td><td>Any associated metadata the client wants to keep.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Request (Version: 1) => group_id generation_id member_id [topics]
group_id => STRING
generation_id => INT32
member_id => STRING
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset commit_timestamp committed_metadata
partition_index => INT32
committed_offset => INT64
commit_timestamp => INT64
committed_metadata => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The unique group identifier.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>topics</td><td>The topics to commit offsets for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>Each partition to commit offsets for.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>committed_offset</td><td>The message offset to be committed.</td></tr>
<tr>
<td>commit_timestamp</td><td>The timestamp of the commit.</td></tr>
<tr>
<td>committed_metadata</td><td>Any associated metadata the client wants to keep.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Request (Version: 2) => group_id generation_id member_id retention_time_ms [topics]
group_id => STRING
generation_id => INT32
member_id => STRING
retention_time_ms => INT64
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset committed_metadata
partition_index => INT32
committed_offset => INT64
committed_metadata => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The unique group identifier.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>retention_time_ms</td><td>The time period in ms to retain the offset.</td></tr>
<tr>
<td>topics</td><td>The topics to commit offsets for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>Each partition to commit offsets for.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>committed_offset</td><td>The message offset to be committed.</td></tr>
<tr>
<td>committed_metadata</td><td>Any associated metadata the client wants to keep.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Request (Version: 3) => group_id generation_id member_id retention_time_ms [topics]
group_id => STRING
generation_id => INT32
member_id => STRING
retention_time_ms => INT64
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset committed_metadata
partition_index => INT32
committed_offset => INT64
committed_metadata => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The unique group identifier.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>retention_time_ms</td><td>The time period in ms to retain the offset.</td></tr>
<tr>
<td>topics</td><td>The topics to commit offsets for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>Each partition to commit offsets for.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>committed_offset</td><td>The message offset to be committed.</td></tr>
<tr>
<td>committed_metadata</td><td>Any associated metadata the client wants to keep.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Request (Version: 4) => group_id generation_id member_id retention_time_ms [topics]
group_id => STRING
generation_id => INT32
member_id => STRING
retention_time_ms => INT64
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset committed_metadata
partition_index => INT32
committed_offset => INT64
committed_metadata => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The unique group identifier.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>retention_time_ms</td><td>The time period in ms to retain the offset.</td></tr>
<tr>
<td>topics</td><td>The topics to commit offsets for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>Each partition to commit offsets for.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>committed_offset</td><td>The message offset to be committed.</td></tr>
<tr>
<td>committed_metadata</td><td>Any associated metadata the client wants to keep.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Request (Version: 5) => group_id generation_id member_id [topics]
group_id => STRING
generation_id => INT32
member_id => STRING
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset committed_metadata
partition_index => INT32
committed_offset => INT64
committed_metadata => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The unique group identifier.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>topics</td><td>The topics to commit offsets for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>Each partition to commit offsets for.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>committed_offset</td><td>The message offset to be committed.</td></tr>
<tr>
<td>committed_metadata</td><td>Any associated metadata the client wants to keep.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Request (Version: 6) => group_id generation_id member_id [topics]
group_id => STRING
generation_id => INT32
member_id => STRING
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset committed_leader_epoch committed_metadata
partition_index => INT32
committed_offset => INT64
committed_leader_epoch => INT32
committed_metadata => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The unique group identifier.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>topics</td><td>The topics to commit offsets for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>Each partition to commit offsets for.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>committed_offset</td><td>The message offset to be committed.</td></tr>
<tr>
<td>committed_leader_epoch</td><td>The leader epoch of this partition.</td></tr>
<tr>
<td>committed_metadata</td><td>Any associated metadata the client wants to keep.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Request (Version: 7) => group_id generation_id member_id group_instance_id [topics]
group_id => STRING
generation_id => INT32
member_id => STRING
group_instance_id => NULLABLE_STRING
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset committed_leader_epoch committed_metadata
partition_index => INT32
committed_offset => INT64
committed_leader_epoch => INT32
committed_metadata => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The unique group identifier.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>group_instance_id</td><td>The unique identifier of the consumer instance provided by end user.</td></tr>
<tr>
<td>topics</td><td>The topics to commit offsets for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>Each partition to commit offsets for.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>committed_offset</td><td>The message offset to be committed.</td></tr>
<tr>
<td>committed_leader_epoch</td><td>The leader epoch of this partition.</td></tr>
<tr>
<td>committed_metadata</td><td>Any associated metadata the client wants to keep.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Request (Version: 8) => group_id generation_id member_id group_instance_id [topics] TAG_BUFFER
group_id => COMPACT_STRING
generation_id => INT32
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index committed_offset committed_leader_epoch committed_metadata TAG_BUFFER
partition_index => INT32
committed_offset => INT64
committed_leader_epoch => INT32
committed_metadata => COMPACT_NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The unique group identifier.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>group_instance_id</td><td>The unique identifier of the consumer instance provided by end user.</td></tr>
<tr>
<td>topics</td><td>The topics to commit offsets for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>Each partition to commit offsets for.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>committed_offset</td><td>The message offset to be committed.</td></tr>
<tr>
<td>committed_leader_epoch</td><td>The leader epoch of this partition.</td></tr>
<tr>
<td>committed_metadata</td><td>Any associated metadata the client wants to keep.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Response (Version: 0) => [topics]
topics => name [partitions]
name => STRING
partitions => partition_index error_code
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The responses for each topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses for each partition in the topic.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Response (Version: 1) => [topics]
topics => name [partitions]
name => STRING
partitions => partition_index error_code
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The responses for each topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses for each partition in the topic.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Response (Version: 2) => [topics]
topics => name [partitions]
name => STRING
partitions => partition_index error_code
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The responses for each topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses for each partition in the topic.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Response (Version: 3) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => name [partitions]
name => STRING
partitions => partition_index error_code
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>The responses for each topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses for each partition in the topic.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Response (Version: 4) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => name [partitions]
name => STRING
partitions => partition_index error_code
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>The responses for each topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses for each partition in the topic.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Response (Version: 5) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => name [partitions]
name => STRING
partitions => partition_index error_code
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>The responses for each topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses for each partition in the topic.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Response (Version: 6) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => name [partitions]
name => STRING
partitions => partition_index error_code
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>The responses for each topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses for each partition in the topic.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Response (Version: 7) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => name [partitions]
name => STRING
partitions => partition_index error_code
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>The responses for each topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses for each partition in the topic.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetCommit Response (Version: 8) => throttle_time_ms [topics] TAG_BUFFER
throttle_time_ms => INT32
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index error_code TAG_BUFFER
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>The responses for each topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses for each partition in the topic.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_OffsetFetch">OffsetFetch API (Key: 9):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">OffsetFetch Request (Version: 0) => group_id [topics]
group_id => STRING
topics => name [partition_indexes]
name => STRING
partition_indexes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group to fetch offsets for.</td></tr>
<tr>
<td>topics</td><td>Each topic we would like to fetch offsets for, or null to fetch offsets for all topics.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partition_indexes</td><td>The partition indexes we would like to fetch offsets for.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetFetch Request (Version: 1) => group_id [topics]
group_id => STRING
topics => name [partition_indexes]
name => STRING
partition_indexes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group to fetch offsets for.</td></tr>
<tr>
<td>topics</td><td>Each topic we would like to fetch offsets for, or null to fetch offsets for all topics.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partition_indexes</td><td>The partition indexes we would like to fetch offsets for.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetFetch Request (Version: 2) => group_id [topics]
group_id => STRING
topics => name [partition_indexes]
name => STRING
partition_indexes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group to fetch offsets for.</td></tr>
<tr>
<td>topics</td><td>Each topic we would like to fetch offsets for, or null to fetch offsets for all topics.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partition_indexes</td><td>The partition indexes we would like to fetch offsets for.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetFetch Request (Version: 3) => group_id [topics]
group_id => STRING
topics => name [partition_indexes]
name => STRING
partition_indexes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group to fetch offsets for.</td></tr>
<tr>
<td>topics</td><td>Each topic we would like to fetch offsets for, or null to fetch offsets for all topics.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partition_indexes</td><td>The partition indexes we would like to fetch offsets for.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetFetch Request (Version: 4) => group_id [topics]
group_id => STRING
topics => name [partition_indexes]
name => STRING
partition_indexes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group to fetch offsets for.</td></tr>
<tr>
<td>topics</td><td>Each topic we would like to fetch offsets for, or null to fetch offsets for all topics.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partition_indexes</td><td>The partition indexes we would like to fetch offsets for.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetFetch Request (Version: 5) => group_id [topics]
group_id => STRING
topics => name [partition_indexes]
name => STRING
partition_indexes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group to fetch offsets for.</td></tr>
<tr>
<td>topics</td><td>Each topic we would like to fetch offsets for, or null to fetch offsets for all topics.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partition_indexes</td><td>The partition indexes we would like to fetch offsets for.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetFetch Request (Version: 6) => group_id [topics] TAG_BUFFER
group_id => COMPACT_STRING
topics => name [partition_indexes] TAG_BUFFER
name => COMPACT_STRING
partition_indexes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group to fetch offsets for.</td></tr>
<tr>
<td>topics</td><td>Each topic we would like to fetch offsets for, or null to fetch offsets for all topics.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partition_indexes</td><td>The partition indexes we would like to fetch offsets for.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">OffsetFetch Response (Version: 0) => [topics]
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset metadata error_code
partition_index => INT32
committed_offset => INT64
metadata => NULLABLE_STRING
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The responses per topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses per partition</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>committed_offset</td><td>The committed message offset.</td></tr>
<tr>
<td>metadata</td><td>The partition metadata.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetFetch Response (Version: 1) => [topics]
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset metadata error_code
partition_index => INT32
committed_offset => INT64
metadata => NULLABLE_STRING
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The responses per topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses per partition</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>committed_offset</td><td>The committed message offset.</td></tr>
<tr>
<td>metadata</td><td>The partition metadata.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetFetch Response (Version: 2) => [topics] error_code
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset metadata error_code
partition_index => INT32
committed_offset => INT64
metadata => NULLABLE_STRING
error_code => INT16
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The responses per topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses per partition</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>committed_offset</td><td>The committed message offset.</td></tr>
<tr>
<td>metadata</td><td>The partition metadata.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>error_code</td><td>The top-level error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetFetch Response (Version: 3) => throttle_time_ms [topics] error_code
throttle_time_ms => INT32
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset metadata error_code
partition_index => INT32
committed_offset => INT64
metadata => NULLABLE_STRING
error_code => INT16
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>The responses per topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses per partition</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>committed_offset</td><td>The committed message offset.</td></tr>
<tr>
<td>metadata</td><td>The partition metadata.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>error_code</td><td>The top-level error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetFetch Response (Version: 4) => throttle_time_ms [topics] error_code
throttle_time_ms => INT32
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset metadata error_code
partition_index => INT32
committed_offset => INT64
metadata => NULLABLE_STRING
error_code => INT16
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>The responses per topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses per partition</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>committed_offset</td><td>The committed message offset.</td></tr>
<tr>
<td>metadata</td><td>The partition metadata.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>error_code</td><td>The top-level error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetFetch Response (Version: 5) => throttle_time_ms [topics] error_code
throttle_time_ms => INT32
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset committed_leader_epoch metadata error_code
partition_index => INT32
committed_offset => INT64
committed_leader_epoch => INT32
metadata => NULLABLE_STRING
error_code => INT16
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>The responses per topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses per partition</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>committed_offset</td><td>The committed message offset.</td></tr>
<tr>
<td>committed_leader_epoch</td><td>The leader epoch.</td></tr>
<tr>
<td>metadata</td><td>The partition metadata.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>error_code</td><td>The top-level error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetFetch Response (Version: 6) => throttle_time_ms [topics] error_code TAG_BUFFER
throttle_time_ms => INT32
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index committed_offset committed_leader_epoch metadata error_code TAG_BUFFER
partition_index => INT32
committed_offset => INT64
committed_leader_epoch => INT32
metadata => COMPACT_NULLABLE_STRING
error_code => INT16
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>The responses per topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses per partition</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>committed_offset</td><td>The committed message offset.</td></tr>
<tr>
<td>committed_leader_epoch</td><td>The leader epoch.</td></tr>
<tr>
<td>metadata</td><td>The partition metadata.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>error_code</td><td>The top-level error code, or 0 if there was no error.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_FindCoordinator">FindCoordinator API (Key: 10):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">FindCoordinator Request (Version: 0) => key
key => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>key</td><td>The coordinator key.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">FindCoordinator Request (Version: 1) => key key_type
key => STRING
key_type => INT8
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>key</td><td>The coordinator key.</td></tr>
<tr>
<td>key_type</td><td>The coordinator key type. (Group, transaction, etc.)</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">FindCoordinator Request (Version: 2) => key key_type
key => STRING
key_type => INT8
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>key</td><td>The coordinator key.</td></tr>
<tr>
<td>key_type</td><td>The coordinator key type. (Group, transaction, etc.)</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">FindCoordinator Request (Version: 3) => key key_type TAG_BUFFER
key => COMPACT_STRING
key_type => INT8
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>key</td><td>The coordinator key.</td></tr>
<tr>
<td>key_type</td><td>The coordinator key type. (Group, transaction, etc.)</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">FindCoordinator Response (Version: 0) => error_code node_id host port
error_code => INT16
node_id => INT32
host => STRING
port => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>node_id</td><td>The node id.</td></tr>
<tr>
<td>host</td><td>The host name.</td></tr>
<tr>
<td>port</td><td>The port.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">FindCoordinator Response (Version: 1) => throttle_time_ms error_code error_message node_id host port
throttle_time_ms => INT32
error_code => INT16
error_message => NULLABLE_STRING
node_id => INT32
host => STRING
port => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>error_message</td><td>The error message, or null if there was no error.</td></tr>
<tr>
<td>node_id</td><td>The node id.</td></tr>
<tr>
<td>host</td><td>The host name.</td></tr>
<tr>
<td>port</td><td>The port.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">FindCoordinator Response (Version: 2) => throttle_time_ms error_code error_message node_id host port
throttle_time_ms => INT32
error_code => INT16
error_message => NULLABLE_STRING
node_id => INT32
host => STRING
port => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>error_message</td><td>The error message, or null if there was no error.</td></tr>
<tr>
<td>node_id</td><td>The node id.</td></tr>
<tr>
<td>host</td><td>The host name.</td></tr>
<tr>
<td>port</td><td>The port.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">FindCoordinator Response (Version: 3) => throttle_time_ms error_code error_message node_id host port TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
node_id => INT32
host => COMPACT_STRING
port => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>error_message</td><td>The error message, or null if there was no error.</td></tr>
<tr>
<td>node_id</td><td>The node id.</td></tr>
<tr>
<td>host</td><td>The host name.</td></tr>
<tr>
<td>port</td><td>The port.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_JoinGroup">JoinGroup API (Key: 11):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">JoinGroup Request (Version: 0) => group_id session_timeout_ms member_id protocol_type [protocols]
group_id => STRING
session_timeout_ms => INT32
member_id => STRING
protocol_type => STRING
protocols => name metadata
name => STRING
metadata => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group identifier.</td></tr>
<tr>
<td>session_timeout_ms</td><td>The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds.</td></tr>
<tr>
<td>member_id</td><td>The member id assigned by the group coordinator.</td></tr>
<tr>
<td>protocol_type</td><td>The unique name the for class of protocols implemented by the group we want to join.</td></tr>
<tr>
<td>protocols</td><td>The list of protocols that the member supports.</td></tr>
<tr>
<td>name</td><td>The protocol name.</td></tr>
<tr>
<td>metadata</td><td>The protocol metadata.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">JoinGroup Request (Version: 1) => group_id session_timeout_ms rebalance_timeout_ms member_id protocol_type [protocols]
group_id => STRING
session_timeout_ms => INT32
rebalance_timeout_ms => INT32
member_id => STRING
protocol_type => STRING
protocols => name metadata
name => STRING
metadata => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group identifier.</td></tr>
<tr>
<td>session_timeout_ms</td><td>The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds.</td></tr>
<tr>
<td>rebalance_timeout_ms</td><td>The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group.</td></tr>
<tr>
<td>member_id</td><td>The member id assigned by the group coordinator.</td></tr>
<tr>
<td>protocol_type</td><td>The unique name the for class of protocols implemented by the group we want to join.</td></tr>
<tr>
<td>protocols</td><td>The list of protocols that the member supports.</td></tr>
<tr>
<td>name</td><td>The protocol name.</td></tr>
<tr>
<td>metadata</td><td>The protocol metadata.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">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
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group identifier.</td></tr>
<tr>
<td>session_timeout_ms</td><td>The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds.</td></tr>
<tr>
<td>rebalance_timeout_ms</td><td>The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group.</td></tr>
<tr>
<td>member_id</td><td>The member id assigned by the group coordinator.</td></tr>
<tr>
<td>protocol_type</td><td>The unique name the for class of protocols implemented by the group we want to join.</td></tr>
<tr>
<td>protocols</td><td>The list of protocols that the member supports.</td></tr>
<tr>
<td>name</td><td>The protocol name.</td></tr>
<tr>
<td>metadata</td><td>The protocol metadata.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">JoinGroup Request (Version: 3) => group_id session_timeout_ms rebalance_timeout_ms member_id protocol_type [protocols]
group_id => STRING
session_timeout_ms => INT32
rebalance_timeout_ms => INT32
member_id => STRING
protocol_type => STRING
protocols => name metadata
name => STRING
metadata => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group identifier.</td></tr>
<tr>
<td>session_timeout_ms</td><td>The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds.</td></tr>
<tr>
<td>rebalance_timeout_ms</td><td>The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group.</td></tr>
<tr>
<td>member_id</td><td>The member id assigned by the group coordinator.</td></tr>
<tr>
<td>protocol_type</td><td>The unique name the for class of protocols implemented by the group we want to join.</td></tr>
<tr>
<td>protocols</td><td>The list of protocols that the member supports.</td></tr>
<tr>
<td>name</td><td>The protocol name.</td></tr>
<tr>
<td>metadata</td><td>The protocol metadata.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">JoinGroup Request (Version: 4) => group_id session_timeout_ms rebalance_timeout_ms member_id protocol_type [protocols]
group_id => STRING
session_timeout_ms => INT32
rebalance_timeout_ms => INT32
member_id => STRING
protocol_type => STRING
protocols => name metadata
name => STRING
metadata => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group identifier.</td></tr>
<tr>
<td>session_timeout_ms</td><td>The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds.</td></tr>
<tr>
<td>rebalance_timeout_ms</td><td>The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group.</td></tr>
<tr>
<td>member_id</td><td>The member id assigned by the group coordinator.</td></tr>
<tr>
<td>protocol_type</td><td>The unique name the for class of protocols implemented by the group we want to join.</td></tr>
<tr>
<td>protocols</td><td>The list of protocols that the member supports.</td></tr>
<tr>
<td>name</td><td>The protocol name.</td></tr>
<tr>
<td>metadata</td><td>The protocol metadata.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">JoinGroup Request (Version: 5) => group_id session_timeout_ms rebalance_timeout_ms member_id group_instance_id protocol_type [protocols]
group_id => STRING
session_timeout_ms => INT32
rebalance_timeout_ms => INT32
member_id => STRING
group_instance_id => NULLABLE_STRING
protocol_type => STRING
protocols => name metadata
name => STRING
metadata => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group identifier.</td></tr>
<tr>
<td>session_timeout_ms</td><td>The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds.</td></tr>
<tr>
<td>rebalance_timeout_ms</td><td>The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group.</td></tr>
<tr>
<td>member_id</td><td>The member id assigned by the group coordinator.</td></tr>
<tr>
<td>group_instance_id</td><td>The unique identifier of the consumer instance provided by end user.</td></tr>
<tr>
<td>protocol_type</td><td>The unique name the for class of protocols implemented by the group we want to join.</td></tr>
<tr>
<td>protocols</td><td>The list of protocols that the member supports.</td></tr>
<tr>
<td>name</td><td>The protocol name.</td></tr>
<tr>
<td>metadata</td><td>The protocol metadata.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">JoinGroup Request (Version: 6) => group_id session_timeout_ms rebalance_timeout_ms member_id group_instance_id protocol_type [protocols] TAG_BUFFER
group_id => COMPACT_STRING
session_timeout_ms => INT32
rebalance_timeout_ms => INT32
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
protocol_type => COMPACT_STRING
protocols => name metadata TAG_BUFFER
name => COMPACT_STRING
metadata => COMPACT_BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group identifier.</td></tr>
<tr>
<td>session_timeout_ms</td><td>The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds.</td></tr>
<tr>
<td>rebalance_timeout_ms</td><td>The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group.</td></tr>
<tr>
<td>member_id</td><td>The member id assigned by the group coordinator.</td></tr>
<tr>
<td>group_instance_id</td><td>The unique identifier of the consumer instance provided by end user.</td></tr>
<tr>
<td>protocol_type</td><td>The unique name the for class of protocols implemented by the group we want to join.</td></tr>
<tr>
<td>protocols</td><td>The list of protocols that the member supports.</td></tr>
<tr>
<td>name</td><td>The protocol name.</td></tr>
<tr>
<td>metadata</td><td>The protocol metadata.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">JoinGroup Response (Version: 0) => error_code generation_id protocol_name leader member_id [members]
error_code => INT16
generation_id => INT32
protocol_name => STRING
leader => STRING
member_id => STRING
members => member_id metadata
member_id => STRING
metadata => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>generation_id</td><td>The generation ID of the group.</td></tr>
<tr>
<td>protocol_name</td><td>The group protocol selected by the coordinator.</td></tr>
<tr>
<td>leader</td><td>The leader of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>members</td><td></td></tr>
<tr>
<td>member_id</td><td>The group member ID.</td></tr>
<tr>
<td>metadata</td><td>The group member metadata.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">JoinGroup Response (Version: 1) => error_code generation_id protocol_name leader member_id [members]
error_code => INT16
generation_id => INT32
protocol_name => STRING
leader => STRING
member_id => STRING
members => member_id metadata
member_id => STRING
metadata => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>generation_id</td><td>The generation ID of the group.</td></tr>
<tr>
<td>protocol_name</td><td>The group protocol selected by the coordinator.</td></tr>
<tr>
<td>leader</td><td>The leader of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>members</td><td></td></tr>
<tr>
<td>member_id</td><td>The group member ID.</td></tr>
<tr>
<td>metadata</td><td>The group member metadata.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">JoinGroup Response (Version: 2) => throttle_time_ms error_code generation_id protocol_name leader member_id [members]
throttle_time_ms => INT32
error_code => INT16
generation_id => INT32
protocol_name => STRING
leader => STRING
member_id => STRING
members => member_id metadata
member_id => STRING
metadata => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>generation_id</td><td>The generation ID of the group.</td></tr>
<tr>
<td>protocol_name</td><td>The group protocol selected by the coordinator.</td></tr>
<tr>
<td>leader</td><td>The leader of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>members</td><td></td></tr>
<tr>
<td>member_id</td><td>The group member ID.</td></tr>
<tr>
<td>metadata</td><td>The group member metadata.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">JoinGroup Response (Version: 3) => throttle_time_ms error_code generation_id protocol_name leader member_id [members]
throttle_time_ms => INT32
error_code => INT16
generation_id => INT32
protocol_name => STRING
leader => STRING
member_id => STRING
members => member_id metadata
member_id => STRING
metadata => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>generation_id</td><td>The generation ID of the group.</td></tr>
<tr>
<td>protocol_name</td><td>The group protocol selected by the coordinator.</td></tr>
<tr>
<td>leader</td><td>The leader of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>members</td><td></td></tr>
<tr>
<td>member_id</td><td>The group member ID.</td></tr>
<tr>
<td>metadata</td><td>The group member metadata.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">JoinGroup Response (Version: 4) => throttle_time_ms error_code generation_id protocol_name leader member_id [members]
throttle_time_ms => INT32
error_code => INT16
generation_id => INT32
protocol_name => STRING
leader => STRING
member_id => STRING
members => member_id metadata
member_id => STRING
metadata => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>generation_id</td><td>The generation ID of the group.</td></tr>
<tr>
<td>protocol_name</td><td>The group protocol selected by the coordinator.</td></tr>
<tr>
<td>leader</td><td>The leader of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>members</td><td></td></tr>
<tr>
<td>member_id</td><td>The group member ID.</td></tr>
<tr>
<td>metadata</td><td>The group member metadata.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">JoinGroup Response (Version: 5) => throttle_time_ms error_code generation_id protocol_name leader member_id [members]
throttle_time_ms => INT32
error_code => INT16
generation_id => INT32
protocol_name => STRING
leader => STRING
member_id => STRING
members => member_id group_instance_id metadata
member_id => STRING
group_instance_id => NULLABLE_STRING
metadata => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>generation_id</td><td>The generation ID of the group.</td></tr>
<tr>
<td>protocol_name</td><td>The group protocol selected by the coordinator.</td></tr>
<tr>
<td>leader</td><td>The leader of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>members</td><td></td></tr>
<tr>
<td>member_id</td><td>The group member ID.</td></tr>
<tr>
<td>group_instance_id</td><td>The unique identifier of the consumer instance provided by end user.</td></tr>
<tr>
<td>metadata</td><td>The group member metadata.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">JoinGroup Response (Version: 6) => throttle_time_ms error_code generation_id protocol_name leader member_id [members] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
generation_id => INT32
protocol_name => COMPACT_STRING
leader => COMPACT_STRING
member_id => COMPACT_STRING
members => member_id group_instance_id metadata TAG_BUFFER
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
metadata => COMPACT_BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>generation_id</td><td>The generation ID of the group.</td></tr>
<tr>
<td>protocol_name</td><td>The group protocol selected by the coordinator.</td></tr>
<tr>
<td>leader</td><td>The leader of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>members</td><td></td></tr>
<tr>
<td>member_id</td><td>The group member ID.</td></tr>
<tr>
<td>group_instance_id</td><td>The unique identifier of the consumer instance provided by end user.</td></tr>
<tr>
<td>metadata</td><td>The group member metadata.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_Heartbeat">Heartbeat API (Key: 12):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">Heartbeat Request (Version: 0) => group_id generation_id member_id
group_id => STRING
generation_id => INT32
member_id => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group id.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Heartbeat Request (Version: 1) => group_id generation_id member_id
group_id => STRING
generation_id => INT32
member_id => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group id.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Heartbeat Request (Version: 2) => group_id generation_id member_id
group_id => STRING
generation_id => INT32
member_id => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group id.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Heartbeat Request (Version: 3) => group_id generation_id member_id group_instance_id
group_id => STRING
generation_id => INT32
member_id => STRING
group_instance_id => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group id.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID.</td></tr>
<tr>
<td>group_instance_id</td><td>The unique identifier of the consumer instance provided by end user.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Heartbeat Request (Version: 4) => group_id generation_id member_id group_instance_id TAG_BUFFER
group_id => COMPACT_STRING
generation_id => INT32
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The group id.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID.</td></tr>
<tr>
<td>group_instance_id</td><td>The unique identifier of the consumer instance provided by end user.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">Heartbeat Response (Version: 0) => error_code
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Heartbeat Response (Version: 1) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Heartbeat Response (Version: 2) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Heartbeat Response (Version: 3) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">Heartbeat Response (Version: 4) => throttle_time_ms error_code TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_LeaveGroup">LeaveGroup API (Key: 13):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">LeaveGroup Request (Version: 0) => group_id member_id
group_id => STRING
member_id => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The ID of the group to leave.</td></tr>
<tr>
<td>member_id</td><td>The member ID to remove from the group.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">LeaveGroup Request (Version: 1) => group_id member_id
group_id => STRING
member_id => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The ID of the group to leave.</td></tr>
<tr>
<td>member_id</td><td>The member ID to remove from the group.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">LeaveGroup Request (Version: 2) => group_id member_id
group_id => STRING
member_id => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The ID of the group to leave.</td></tr>
<tr>
<td>member_id</td><td>The member ID to remove from the group.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">LeaveGroup Request (Version: 3) => group_id [members]
group_id => STRING
members => member_id group_instance_id
member_id => STRING
group_instance_id => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The ID of the group to leave.</td></tr>
<tr>
<td>members</td><td>List of leaving member identities.</td></tr>
<tr>
<td>member_id</td><td>The member ID to remove from the group.</td></tr>
<tr>
<td>group_instance_id</td><td>The group instance ID to remove from the group.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">LeaveGroup Request (Version: 4) => group_id [members] TAG_BUFFER
group_id => COMPACT_STRING
members => member_id group_instance_id TAG_BUFFER
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The ID of the group to leave.</td></tr>
<tr>
<td>members</td><td>List of leaving member identities.</td></tr>
<tr>
<td>member_id</td><td>The member ID to remove from the group.</td></tr>
<tr>
<td>group_instance_id</td><td>The group instance ID to remove from the group.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">LeaveGroup Response (Version: 0) => error_code
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">LeaveGroup Response (Version: 1) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">LeaveGroup Response (Version: 2) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">LeaveGroup Response (Version: 3) => throttle_time_ms error_code [members]
throttle_time_ms => INT32
error_code => INT16
members => member_id group_instance_id error_code
member_id => STRING
group_instance_id => NULLABLE_STRING
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>members</td><td>List of leaving member responses.</td></tr>
<tr>
<td>member_id</td><td>The member ID to remove from the group.</td></tr>
<tr>
<td>group_instance_id</td><td>The group instance ID to remove from the group.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">LeaveGroup Response (Version: 4) => throttle_time_ms error_code [members] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
members => member_id group_instance_id error_code TAG_BUFFER
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>members</td><td>List of leaving member responses.</td></tr>
<tr>
<td>member_id</td><td>The member ID to remove from the group.</td></tr>
<tr>
<td>group_instance_id</td><td>The group instance ID to remove from the group.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_SyncGroup">SyncGroup API (Key: 14):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">SyncGroup Request (Version: 0) => group_id generation_id member_id [assignments]
group_id => STRING
generation_id => INT32
member_id => STRING
assignments => member_id assignment
member_id => STRING
assignment => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The unique group identifier.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group.</td></tr>
<tr>
<td>assignments</td><td>Each assignment.</td></tr>
<tr>
<td>member_id</td><td>The ID of the member to assign.</td></tr>
<tr>
<td>assignment</td><td>The member assignment.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">SyncGroup Request (Version: 1) => group_id generation_id member_id [assignments]
group_id => STRING
generation_id => INT32
member_id => STRING
assignments => member_id assignment
member_id => STRING
assignment => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The unique group identifier.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group.</td></tr>
<tr>
<td>assignments</td><td>Each assignment.</td></tr>
<tr>
<td>member_id</td><td>The ID of the member to assign.</td></tr>
<tr>
<td>assignment</td><td>The member assignment.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">SyncGroup Request (Version: 2) => group_id generation_id member_id [assignments]
group_id => STRING
generation_id => INT32
member_id => STRING
assignments => member_id assignment
member_id => STRING
assignment => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The unique group identifier.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group.</td></tr>
<tr>
<td>assignments</td><td>Each assignment.</td></tr>
<tr>
<td>member_id</td><td>The ID of the member to assign.</td></tr>
<tr>
<td>assignment</td><td>The member assignment.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">SyncGroup Request (Version: 3) => group_id generation_id member_id group_instance_id [assignments]
group_id => STRING
generation_id => INT32
member_id => STRING
group_instance_id => NULLABLE_STRING
assignments => member_id assignment
member_id => STRING
assignment => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The unique group identifier.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group.</td></tr>
<tr>
<td>group_instance_id</td><td>The unique identifier of the consumer instance provided by end user.</td></tr>
<tr>
<td>assignments</td><td>Each assignment.</td></tr>
<tr>
<td>member_id</td><td>The ID of the member to assign.</td></tr>
<tr>
<td>assignment</td><td>The member assignment.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">SyncGroup Request (Version: 4) => group_id generation_id member_id group_instance_id [assignments] TAG_BUFFER
group_id => COMPACT_STRING
generation_id => INT32
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
assignments => member_id assignment TAG_BUFFER
member_id => COMPACT_STRING
assignment => COMPACT_BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The unique group identifier.</td></tr>
<tr>
<td>generation_id</td><td>The generation of the group.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group.</td></tr>
<tr>
<td>group_instance_id</td><td>The unique identifier of the consumer instance provided by end user.</td></tr>
<tr>
<td>assignments</td><td>Each assignment.</td></tr>
<tr>
<td>member_id</td><td>The ID of the member to assign.</td></tr>
<tr>
<td>assignment</td><td>The member assignment.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">SyncGroup Response (Version: 0) => error_code assignment
error_code => INT16
assignment => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>assignment</td><td>The member assignment.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">SyncGroup Response (Version: 1) => throttle_time_ms error_code assignment
throttle_time_ms => INT32
error_code => INT16
assignment => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>assignment</td><td>The member assignment.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">SyncGroup Response (Version: 2) => throttle_time_ms error_code assignment
throttle_time_ms => INT32
error_code => INT16
assignment => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>assignment</td><td>The member assignment.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">SyncGroup Response (Version: 3) => throttle_time_ms error_code assignment
throttle_time_ms => INT32
error_code => INT16
assignment => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>assignment</td><td>The member assignment.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">SyncGroup Response (Version: 4) => throttle_time_ms error_code assignment TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
assignment => COMPACT_BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>assignment</td><td>The member assignment.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_DescribeGroups">DescribeGroups API (Key: 15):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">DescribeGroups Request (Version: 0) => [groups]
groups => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>groups</td><td>The names of the groups to describe</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeGroups Request (Version: 1) => [groups]
groups => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>groups</td><td>The names of the groups to describe</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeGroups Request (Version: 2) => [groups]
groups => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>groups</td><td>The names of the groups to describe</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeGroups Request (Version: 3) => [groups] include_authorized_operations
groups => STRING
include_authorized_operations => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>groups</td><td>The names of the groups to describe</td></tr>
<tr>
<td>include_authorized_operations</td><td>Whether to include authorized operations.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeGroups Request (Version: 4) => [groups] include_authorized_operations
groups => STRING
include_authorized_operations => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>groups</td><td>The names of the groups to describe</td></tr>
<tr>
<td>include_authorized_operations</td><td>Whether to include authorized operations.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeGroups Request (Version: 5) => [groups] include_authorized_operations TAG_BUFFER
groups => COMPACT_STRING
include_authorized_operations => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>groups</td><td>The names of the groups to describe</td></tr>
<tr>
<td>include_authorized_operations</td><td>Whether to include authorized operations.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">DescribeGroups Response (Version: 0) => [groups]
groups => error_code group_id group_state protocol_type protocol_data [members]
error_code => INT16
group_id => STRING
group_state => STRING
protocol_type => STRING
protocol_data => STRING
members => member_id client_id client_host member_metadata member_assignment
member_id => STRING
client_id => STRING
client_host => STRING
member_metadata => BYTES
member_assignment => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>groups</td><td>Each described group.</td></tr>
<tr>
<td>error_code</td><td>The describe error, or 0 if there was no error.</td></tr>
<tr>
<td>group_id</td><td>The group ID string.</td></tr>
<tr>
<td>group_state</td><td>The group state string, or the empty string.</td></tr>
<tr>
<td>protocol_type</td><td>The group protocol type, or the empty string.</td></tr>
<tr>
<td>protocol_data</td><td>The group protocol data, or the empty string.</td></tr>
<tr>
<td>members</td><td>The group members.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>client_id</td><td>The client ID used in the member's latest join group request.</td></tr>
<tr>
<td>client_host</td><td>The client host.</td></tr>
<tr>
<td>member_metadata</td><td>The metadata corresponding to the current group protocol in use.</td></tr>
<tr>
<td>member_assignment</td><td>The current assignment provided by the group leader.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeGroups Response (Version: 1) => throttle_time_ms [groups]
throttle_time_ms => INT32
groups => error_code group_id group_state protocol_type protocol_data [members]
error_code => INT16
group_id => STRING
group_state => STRING
protocol_type => STRING
protocol_data => STRING
members => member_id client_id client_host member_metadata member_assignment
member_id => STRING
client_id => STRING
client_host => STRING
member_metadata => BYTES
member_assignment => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>groups</td><td>Each described group.</td></tr>
<tr>
<td>error_code</td><td>The describe error, or 0 if there was no error.</td></tr>
<tr>
<td>group_id</td><td>The group ID string.</td></tr>
<tr>
<td>group_state</td><td>The group state string, or the empty string.</td></tr>
<tr>
<td>protocol_type</td><td>The group protocol type, or the empty string.</td></tr>
<tr>
<td>protocol_data</td><td>The group protocol data, or the empty string.</td></tr>
<tr>
<td>members</td><td>The group members.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>client_id</td><td>The client ID used in the member's latest join group request.</td></tr>
<tr>
<td>client_host</td><td>The client host.</td></tr>
<tr>
<td>member_metadata</td><td>The metadata corresponding to the current group protocol in use.</td></tr>
<tr>
<td>member_assignment</td><td>The current assignment provided by the group leader.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeGroups Response (Version: 2) => throttle_time_ms [groups]
throttle_time_ms => INT32
groups => error_code group_id group_state protocol_type protocol_data [members]
error_code => INT16
group_id => STRING
group_state => STRING
protocol_type => STRING
protocol_data => STRING
members => member_id client_id client_host member_metadata member_assignment
member_id => STRING
client_id => STRING
client_host => STRING
member_metadata => BYTES
member_assignment => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>groups</td><td>Each described group.</td></tr>
<tr>
<td>error_code</td><td>The describe error, or 0 if there was no error.</td></tr>
<tr>
<td>group_id</td><td>The group ID string.</td></tr>
<tr>
<td>group_state</td><td>The group state string, or the empty string.</td></tr>
<tr>
<td>protocol_type</td><td>The group protocol type, or the empty string.</td></tr>
<tr>
<td>protocol_data</td><td>The group protocol data, or the empty string.</td></tr>
<tr>
<td>members</td><td>The group members.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>client_id</td><td>The client ID used in the member's latest join group request.</td></tr>
<tr>
<td>client_host</td><td>The client host.</td></tr>
<tr>
<td>member_metadata</td><td>The metadata corresponding to the current group protocol in use.</td></tr>
<tr>
<td>member_assignment</td><td>The current assignment provided by the group leader.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeGroups Response (Version: 3) => throttle_time_ms [groups]
throttle_time_ms => INT32
groups => error_code group_id group_state protocol_type protocol_data [members] authorized_operations
error_code => INT16
group_id => STRING
group_state => STRING
protocol_type => STRING
protocol_data => STRING
members => member_id client_id client_host member_metadata member_assignment
member_id => STRING
client_id => STRING
client_host => STRING
member_metadata => BYTES
member_assignment => BYTES
authorized_operations => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>groups</td><td>Each described group.</td></tr>
<tr>
<td>error_code</td><td>The describe error, or 0 if there was no error.</td></tr>
<tr>
<td>group_id</td><td>The group ID string.</td></tr>
<tr>
<td>group_state</td><td>The group state string, or the empty string.</td></tr>
<tr>
<td>protocol_type</td><td>The group protocol type, or the empty string.</td></tr>
<tr>
<td>protocol_data</td><td>The group protocol data, or the empty string.</td></tr>
<tr>
<td>members</td><td>The group members.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>client_id</td><td>The client ID used in the member's latest join group request.</td></tr>
<tr>
<td>client_host</td><td>The client host.</td></tr>
<tr>
<td>member_metadata</td><td>The metadata corresponding to the current group protocol in use.</td></tr>
<tr>
<td>member_assignment</td><td>The current assignment provided by the group leader.</td></tr>
<tr>
<td>authorized_operations</td><td>32-bit bitfield to represent authorized operations for this group.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeGroups Response (Version: 4) => throttle_time_ms [groups]
throttle_time_ms => INT32
groups => error_code group_id group_state protocol_type protocol_data [members] authorized_operations
error_code => INT16
group_id => STRING
group_state => STRING
protocol_type => STRING
protocol_data => STRING
members => member_id group_instance_id client_id client_host member_metadata member_assignment
member_id => STRING
group_instance_id => NULLABLE_STRING
client_id => STRING
client_host => STRING
member_metadata => BYTES
member_assignment => BYTES
authorized_operations => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>groups</td><td>Each described group.</td></tr>
<tr>
<td>error_code</td><td>The describe error, or 0 if there was no error.</td></tr>
<tr>
<td>group_id</td><td>The group ID string.</td></tr>
<tr>
<td>group_state</td><td>The group state string, or the empty string.</td></tr>
<tr>
<td>protocol_type</td><td>The group protocol type, or the empty string.</td></tr>
<tr>
<td>protocol_data</td><td>The group protocol data, or the empty string.</td></tr>
<tr>
<td>members</td><td>The group members.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>group_instance_id</td><td>The unique identifier of the consumer instance provided by end user.</td></tr>
<tr>
<td>client_id</td><td>The client ID used in the member's latest join group request.</td></tr>
<tr>
<td>client_host</td><td>The client host.</td></tr>
<tr>
<td>member_metadata</td><td>The metadata corresponding to the current group protocol in use.</td></tr>
<tr>
<td>member_assignment</td><td>The current assignment provided by the group leader.</td></tr>
<tr>
<td>authorized_operations</td><td>32-bit bitfield to represent authorized operations for this group.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeGroups Response (Version: 5) => throttle_time_ms [groups] TAG_BUFFER
throttle_time_ms => INT32
groups => error_code group_id group_state protocol_type protocol_data [members] authorized_operations TAG_BUFFER
error_code => INT16
group_id => COMPACT_STRING
group_state => COMPACT_STRING
protocol_type => COMPACT_STRING
protocol_data => COMPACT_STRING
members => member_id group_instance_id client_id client_host member_metadata member_assignment TAG_BUFFER
member_id => COMPACT_STRING
group_instance_id => COMPACT_NULLABLE_STRING
client_id => COMPACT_STRING
client_host => COMPACT_STRING
member_metadata => COMPACT_BYTES
member_assignment => COMPACT_BYTES
authorized_operations => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>groups</td><td>Each described group.</td></tr>
<tr>
<td>error_code</td><td>The describe error, or 0 if there was no error.</td></tr>
<tr>
<td>group_id</td><td>The group ID string.</td></tr>
<tr>
<td>group_state</td><td>The group state string, or the empty string.</td></tr>
<tr>
<td>protocol_type</td><td>The group protocol type, or the empty string.</td></tr>
<tr>
<td>protocol_data</td><td>The group protocol data, or the empty string.</td></tr>
<tr>
<td>members</td><td>The group members.</td></tr>
<tr>
<td>member_id</td><td>The member ID assigned by the group coordinator.</td></tr>
<tr>
<td>group_instance_id</td><td>The unique identifier of the consumer instance provided by end user.</td></tr>
<tr>
<td>client_id</td><td>The client ID used in the member's latest join group request.</td></tr>
<tr>
<td>client_host</td><td>The client host.</td></tr>
<tr>
<td>member_metadata</td><td>The metadata corresponding to the current group protocol in use.</td></tr>
<tr>
<td>member_assignment</td><td>The current assignment provided by the group leader.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>authorized_operations</td><td>32-bit bitfield to represent authorized operations for this group.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_ListGroups">ListGroups API (Key: 16):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">ListGroups Request (Version: 0) =>
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr></table>
</p>
<p><pre class="line-numbers"><code class="language-java">ListGroups Request (Version: 1) =>
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr></table>
</p>
<p><pre class="line-numbers"><code class="language-java">ListGroups Request (Version: 2) =>
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr></table>
</p>
<p><pre class="line-numbers"><code class="language-java">ListGroups Request (Version: 3) => TAG_BUFFER
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">ListGroups Response (Version: 0) => error_code [groups]
error_code => INT16
groups => group_id protocol_type
group_id => STRING
protocol_type => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>groups</td><td>Each group in the response.</td></tr>
<tr>
<td>group_id</td><td>The group ID.</td></tr>
<tr>
<td>protocol_type</td><td>The group protocol type.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ListGroups Response (Version: 1) => throttle_time_ms error_code [groups]
throttle_time_ms => INT32
error_code => INT16
groups => group_id protocol_type
group_id => STRING
protocol_type => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>groups</td><td>Each group in the response.</td></tr>
<tr>
<td>group_id</td><td>The group ID.</td></tr>
<tr>
<td>protocol_type</td><td>The group protocol type.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ListGroups Response (Version: 2) => throttle_time_ms error_code [groups]
throttle_time_ms => INT32
error_code => INT16
groups => group_id protocol_type
group_id => STRING
protocol_type => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>groups</td><td>Each group in the response.</td></tr>
<tr>
<td>group_id</td><td>The group ID.</td></tr>
<tr>
<td>protocol_type</td><td>The group protocol type.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ListGroups Response (Version: 3) => throttle_time_ms error_code [groups] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
groups => group_id protocol_type TAG_BUFFER
group_id => COMPACT_STRING
protocol_type => COMPACT_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>groups</td><td>Each group in the response.</td></tr>
<tr>
<td>group_id</td><td>The group ID.</td></tr>
<tr>
<td>protocol_type</td><td>The group protocol type.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_SaslHandshake">SaslHandshake API (Key: 17):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">SaslHandshake Request (Version: 0) => mechanism
mechanism => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>mechanism</td><td>The SASL mechanism chosen by the client.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">SaslHandshake Request (Version: 1) => mechanism
mechanism => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>mechanism</td><td>The SASL mechanism chosen by the client.</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">SaslHandshake Response (Version: 0) => error_code [mechanisms]
error_code => INT16
mechanisms => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>mechanisms</td><td>The mechanisms enabled in the server.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">SaslHandshake Response (Version: 1) => error_code [mechanisms]
error_code => INT16
mechanisms => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>mechanisms</td><td>The mechanisms enabled in the server.</td></tr>
</table>
</p>
<h5><a name="The_Messages_ApiVersions">ApiVersions API (Key: 18):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">ApiVersions Request (Version: 0) =>
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr></table>
</p>
<p><pre class="line-numbers"><code class="language-java">ApiVersions Request (Version: 1) =>
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr></table>
</p>
<p><pre class="line-numbers"><code class="language-java">ApiVersions Request (Version: 2) =>
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr></table>
</p>
<p><pre class="line-numbers"><code class="language-java">ApiVersions Request (Version: 3) => client_software_name client_software_version TAG_BUFFER
client_software_name => COMPACT_STRING
client_software_version => COMPACT_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>client_software_name</td><td>The name of the client.</td></tr>
<tr>
<td>client_software_version</td><td>The version of the client.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">ApiVersions Response (Version: 0) => error_code [api_keys]
error_code => INT16
api_keys => api_key min_version max_version
api_key => INT16
min_version => INT16
max_version => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The top-level error code.</td></tr>
<tr>
<td>api_keys</td><td>The APIs supported by the broker.</td></tr>
<tr>
<td>api_key</td><td>The API index.</td></tr>
<tr>
<td>min_version</td><td>The minimum supported version, inclusive.</td></tr>
<tr>
<td>max_version</td><td>The maximum supported version, inclusive.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ApiVersions Response (Version: 1) => error_code [api_keys] throttle_time_ms
error_code => INT16
api_keys => api_key min_version max_version
api_key => INT16
min_version => INT16
max_version => INT16
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The top-level error code.</td></tr>
<tr>
<td>api_keys</td><td>The APIs supported by the broker.</td></tr>
<tr>
<td>api_key</td><td>The API index.</td></tr>
<tr>
<td>min_version</td><td>The minimum supported version, inclusive.</td></tr>
<tr>
<td>max_version</td><td>The maximum supported version, inclusive.</td></tr>
<tr>
<td>throttle_time_ms</td><td>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.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ApiVersions Response (Version: 2) => error_code [api_keys] throttle_time_ms
error_code => INT16
api_keys => api_key min_version max_version
api_key => INT16
min_version => INT16
max_version => INT16
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The top-level error code.</td></tr>
<tr>
<td>api_keys</td><td>The APIs supported by the broker.</td></tr>
<tr>
<td>api_key</td><td>The API index.</td></tr>
<tr>
<td>min_version</td><td>The minimum supported version, inclusive.</td></tr>
<tr>
<td>max_version</td><td>The maximum supported version, inclusive.</td></tr>
<tr>
<td>throttle_time_ms</td><td>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.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ApiVersions Response (Version: 3) => error_code [api_keys] throttle_time_ms TAG_BUFFER
error_code => INT16
api_keys => api_key min_version max_version TAG_BUFFER
api_key => INT16
min_version => INT16
max_version => INT16
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The top-level error code.</td></tr>
<tr>
<td>api_keys</td><td>The APIs supported by the broker.</td></tr>
<tr>
<td>api_key</td><td>The API index.</td></tr>
<tr>
<td>min_version</td><td>The minimum supported version, inclusive.</td></tr>
<tr>
<td>max_version</td><td>The maximum supported version, inclusive.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_CreateTopics">CreateTopics API (Key: 19):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">CreateTopics Request (Version: 0) => [topics] timeout_ms
topics => name num_partitions replication_factor [assignments] [configs]
name => STRING
num_partitions => INT32
replication_factor => INT16
assignments => partition_index [broker_ids]
partition_index => INT32
broker_ids => INT32
configs => name value
name => STRING
value => NULLABLE_STRING
timeout_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The topics to create.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>num_partitions</td><td>The number of partitions to create in the topic, or -1 if we are either specifying a manual partition assignment or using the default partitions.</td></tr>
<tr>
<td>replication_factor</td><td>The number of replicas to create for each partition in the topic, or -1 if we are either specifying a manual partition assignment or using the default replication factor.</td></tr>
<tr>
<td>assignments</td><td>The manual partition assignment, or the empty array if we are using automatic assignment.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>broker_ids</td><td>The brokers to place the partition on.</td></tr>
<tr>
<td>configs</td><td>The custom topic configurations to set.</td></tr>
<tr>
<td>name</td><td>The configuration name.</td></tr>
<tr>
<td>value</td><td>The configuration value.</td></tr>
<tr>
<td>timeout_ms</td><td>How long to wait in milliseconds before timing out the request.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreateTopics Request (Version: 1) => [topics] timeout_ms validate_only
topics => name num_partitions replication_factor [assignments] [configs]
name => STRING
num_partitions => INT32
replication_factor => INT16
assignments => partition_index [broker_ids]
partition_index => INT32
broker_ids => INT32
configs => name value
name => STRING
value => NULLABLE_STRING
timeout_ms => INT32
validate_only => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The topics to create.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>num_partitions</td><td>The number of partitions to create in the topic, or -1 if we are either specifying a manual partition assignment or using the default partitions.</td></tr>
<tr>
<td>replication_factor</td><td>The number of replicas to create for each partition in the topic, or -1 if we are either specifying a manual partition assignment or using the default replication factor.</td></tr>
<tr>
<td>assignments</td><td>The manual partition assignment, or the empty array if we are using automatic assignment.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>broker_ids</td><td>The brokers to place the partition on.</td></tr>
<tr>
<td>configs</td><td>The custom topic configurations to set.</td></tr>
<tr>
<td>name</td><td>The configuration name.</td></tr>
<tr>
<td>value</td><td>The configuration value.</td></tr>
<tr>
<td>timeout_ms</td><td>How long to wait in milliseconds before timing out the request.</td></tr>
<tr>
<td>validate_only</td><td>If true, check that the topics can be created as specified, but don't create anything.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreateTopics Request (Version: 2) => [topics] timeout_ms validate_only
topics => name num_partitions replication_factor [assignments] [configs]
name => STRING
num_partitions => INT32
replication_factor => INT16
assignments => partition_index [broker_ids]
partition_index => INT32
broker_ids => INT32
configs => name value
name => STRING
value => NULLABLE_STRING
timeout_ms => INT32
validate_only => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The topics to create.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>num_partitions</td><td>The number of partitions to create in the topic, or -1 if we are either specifying a manual partition assignment or using the default partitions.</td></tr>
<tr>
<td>replication_factor</td><td>The number of replicas to create for each partition in the topic, or -1 if we are either specifying a manual partition assignment or using the default replication factor.</td></tr>
<tr>
<td>assignments</td><td>The manual partition assignment, or the empty array if we are using automatic assignment.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>broker_ids</td><td>The brokers to place the partition on.</td></tr>
<tr>
<td>configs</td><td>The custom topic configurations to set.</td></tr>
<tr>
<td>name</td><td>The configuration name.</td></tr>
<tr>
<td>value</td><td>The configuration value.</td></tr>
<tr>
<td>timeout_ms</td><td>How long to wait in milliseconds before timing out the request.</td></tr>
<tr>
<td>validate_only</td><td>If true, check that the topics can be created as specified, but don't create anything.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreateTopics Request (Version: 3) => [topics] timeout_ms validate_only
topics => name num_partitions replication_factor [assignments] [configs]
name => STRING
num_partitions => INT32
replication_factor => INT16
assignments => partition_index [broker_ids]
partition_index => INT32
broker_ids => INT32
configs => name value
name => STRING
value => NULLABLE_STRING
timeout_ms => INT32
validate_only => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The topics to create.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>num_partitions</td><td>The number of partitions to create in the topic, or -1 if we are either specifying a manual partition assignment or using the default partitions.</td></tr>
<tr>
<td>replication_factor</td><td>The number of replicas to create for each partition in the topic, or -1 if we are either specifying a manual partition assignment or using the default replication factor.</td></tr>
<tr>
<td>assignments</td><td>The manual partition assignment, or the empty array if we are using automatic assignment.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>broker_ids</td><td>The brokers to place the partition on.</td></tr>
<tr>
<td>configs</td><td>The custom topic configurations to set.</td></tr>
<tr>
<td>name</td><td>The configuration name.</td></tr>
<tr>
<td>value</td><td>The configuration value.</td></tr>
<tr>
<td>timeout_ms</td><td>How long to wait in milliseconds before timing out the request.</td></tr>
<tr>
<td>validate_only</td><td>If true, check that the topics can be created as specified, but don't create anything.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreateTopics Request (Version: 4) => [topics] timeout_ms validate_only
topics => name num_partitions replication_factor [assignments] [configs]
name => STRING
num_partitions => INT32
replication_factor => INT16
assignments => partition_index [broker_ids]
partition_index => INT32
broker_ids => INT32
configs => name value
name => STRING
value => NULLABLE_STRING
timeout_ms => INT32
validate_only => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The topics to create.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>num_partitions</td><td>The number of partitions to create in the topic, or -1 if we are either specifying a manual partition assignment or using the default partitions.</td></tr>
<tr>
<td>replication_factor</td><td>The number of replicas to create for each partition in the topic, or -1 if we are either specifying a manual partition assignment or using the default replication factor.</td></tr>
<tr>
<td>assignments</td><td>The manual partition assignment, or the empty array if we are using automatic assignment.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>broker_ids</td><td>The brokers to place the partition on.</td></tr>
<tr>
<td>configs</td><td>The custom topic configurations to set.</td></tr>
<tr>
<td>name</td><td>The configuration name.</td></tr>
<tr>
<td>value</td><td>The configuration value.</td></tr>
<tr>
<td>timeout_ms</td><td>How long to wait in milliseconds before timing out the request.</td></tr>
<tr>
<td>validate_only</td><td>If true, check that the topics can be created as specified, but don't create anything.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreateTopics Request (Version: 5) => [topics] timeout_ms validate_only TAG_BUFFER
topics => name num_partitions replication_factor [assignments] [configs] TAG_BUFFER
name => COMPACT_STRING
num_partitions => INT32
replication_factor => INT16
assignments => partition_index [broker_ids] TAG_BUFFER
partition_index => INT32
broker_ids => INT32
configs => name value TAG_BUFFER
name => COMPACT_STRING
value => COMPACT_NULLABLE_STRING
timeout_ms => INT32
validate_only => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>The topics to create.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>num_partitions</td><td>The number of partitions to create in the topic, or -1 if we are either specifying a manual partition assignment or using the default partitions.</td></tr>
<tr>
<td>replication_factor</td><td>The number of replicas to create for each partition in the topic, or -1 if we are either specifying a manual partition assignment or using the default replication factor.</td></tr>
<tr>
<td>assignments</td><td>The manual partition assignment, or the empty array if we are using automatic assignment.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>broker_ids</td><td>The brokers to place the partition on.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>configs</td><td>The custom topic configurations to set.</td></tr>
<tr>
<td>name</td><td>The configuration name.</td></tr>
<tr>
<td>value</td><td>The configuration value.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>timeout_ms</td><td>How long to wait in milliseconds before timing out the request.</td></tr>
<tr>
<td>validate_only</td><td>If true, check that the topics can be created as specified, but don't create anything.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">CreateTopics Response (Version: 0) => [topics]
topics => name error_code
name => STRING
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>Results for each topic we tried to create.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreateTopics Response (Version: 1) => [topics]
topics => name error_code error_message
name => STRING
error_code => INT16
error_message => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>Results for each topic we tried to create.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>error_message</td><td>The error message, or null if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreateTopics Response (Version: 2) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => name error_code error_message
name => STRING
error_code => INT16
error_message => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>Results for each topic we tried to create.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>error_message</td><td>The error message, or null if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreateTopics Response (Version: 3) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => name error_code error_message
name => STRING
error_code => INT16
error_message => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>Results for each topic we tried to create.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>error_message</td><td>The error message, or null if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreateTopics Response (Version: 4) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => name error_code error_message
name => STRING
error_code => INT16
error_message => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>Results for each topic we tried to create.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>error_message</td><td>The error message, or null if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreateTopics Response (Version: 5) => throttle_time_ms [topics] TAG_BUFFER
throttle_time_ms => INT32
topics => name error_code error_message num_partitions replication_factor [configs] TAG_BUFFER
name => COMPACT_STRING
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
num_partitions => INT32
replication_factor => INT16
configs => name value read_only config_source is_sensitive TAG_BUFFER
name => COMPACT_STRING
value => COMPACT_NULLABLE_STRING
read_only => BOOLEAN
config_source => INT8
is_sensitive => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>Results for each topic we tried to create.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>error_message</td><td>The error message, or null if there was no error.</td></tr>
<tr>
<td>num_partitions</td><td>Number of partitions of the topic.</td></tr>
<tr>
<td>replication_factor</td><td>Replicator factor of the topic.</td></tr>
<tr>
<td>configs</td><td>Configuration of the topic.</td></tr>
<tr>
<td>name</td><td>The configuration name.</td></tr>
<tr>
<td>value</td><td>The configuration value.</td></tr>
<tr>
<td>read_only</td><td>True if the configuration is read-only.</td></tr>
<tr>
<td>config_source</td><td>The configuration source.</td></tr>
<tr>
<td>is_sensitive</td><td>True if this configuration is sensitive.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_DeleteTopics">DeleteTopics API (Key: 20):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">DeleteTopics Request (Version: 0) => [topic_names] timeout_ms
topic_names => STRING
timeout_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topic_names</td><td>The names of the topics to delete</td></tr>
<tr>
<td>timeout_ms</td><td>The length of time in milliseconds to wait for the deletions to complete.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DeleteTopics Request (Version: 1) => [topic_names] timeout_ms
topic_names => STRING
timeout_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topic_names</td><td>The names of the topics to delete</td></tr>
<tr>
<td>timeout_ms</td><td>The length of time in milliseconds to wait for the deletions to complete.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DeleteTopics Request (Version: 2) => [topic_names] timeout_ms
topic_names => STRING
timeout_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topic_names</td><td>The names of the topics to delete</td></tr>
<tr>
<td>timeout_ms</td><td>The length of time in milliseconds to wait for the deletions to complete.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DeleteTopics Request (Version: 3) => [topic_names] timeout_ms
topic_names => STRING
timeout_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topic_names</td><td>The names of the topics to delete</td></tr>
<tr>
<td>timeout_ms</td><td>The length of time in milliseconds to wait for the deletions to complete.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DeleteTopics Request (Version: 4) => [topic_names] timeout_ms TAG_BUFFER
topic_names => COMPACT_STRING
timeout_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topic_names</td><td>The names of the topics to delete</td></tr>
<tr>
<td>timeout_ms</td><td>The length of time in milliseconds to wait for the deletions to complete.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">DeleteTopics Response (Version: 0) => [responses]
responses => name error_code
name => STRING
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>responses</td><td>The results for each topic we tried to delete.</td></tr>
<tr>
<td>name</td><td>The topic name</td></tr>
<tr>
<td>error_code</td><td>The deletion error, or 0 if the deletion succeeded.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DeleteTopics Response (Version: 1) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => name error_code
name => STRING
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>responses</td><td>The results for each topic we tried to delete.</td></tr>
<tr>
<td>name</td><td>The topic name</td></tr>
<tr>
<td>error_code</td><td>The deletion error, or 0 if the deletion succeeded.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DeleteTopics Response (Version: 2) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => name error_code
name => STRING
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>responses</td><td>The results for each topic we tried to delete.</td></tr>
<tr>
<td>name</td><td>The topic name</td></tr>
<tr>
<td>error_code</td><td>The deletion error, or 0 if the deletion succeeded.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DeleteTopics Response (Version: 3) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => name error_code
name => STRING
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>responses</td><td>The results for each topic we tried to delete.</td></tr>
<tr>
<td>name</td><td>The topic name</td></tr>
<tr>
<td>error_code</td><td>The deletion error, or 0 if the deletion succeeded.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DeleteTopics Response (Version: 4) => throttle_time_ms [responses] TAG_BUFFER
throttle_time_ms => INT32
responses => name error_code TAG_BUFFER
name => COMPACT_STRING
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>responses</td><td>The results for each topic we tried to delete.</td></tr>
<tr>
<td>name</td><td>The topic name</td></tr>
<tr>
<td>error_code</td><td>The deletion error, or 0 if the deletion succeeded.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_DeleteRecords">DeleteRecords API (Key: 21):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">DeleteRecords Request (Version: 0) => [topics] timeout
topics => topic [partitions]
topic => STRING
partitions => partition offset
partition => INT32
offset => INT64
timeout => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>offset</td><td>The offset before which the messages will be deleted. -1 means high-watermark for the partition.</td></tr>
<tr>
<td>timeout</td><td>The maximum time to await a response in ms.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DeleteRecords Request (Version: 1) => [topics] timeout
topics => topic [partitions]
topic => STRING
partitions => partition offset
partition => INT32
offset => INT64
timeout => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>offset</td><td>The offset before which the messages will be deleted. -1 means high-watermark for the partition.</td></tr>
<tr>
<td>timeout</td><td>The maximum time to await a response in ms.</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">DeleteRecords Response (Version: 0) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => topic [partitions]
topic => STRING
partitions => partition low_watermark error_code
partition => INT32
low_watermark => INT64
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>topics</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>low_watermark</td><td>Smallest available offset of all live replicas</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DeleteRecords Response (Version: 1) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => topic [partitions]
topic => STRING
partitions => partition low_watermark error_code
partition => INT32
low_watermark => INT64
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>topics</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>low_watermark</td><td>Smallest available offset of all live replicas</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
</table>
</p>
<h5><a name="The_Messages_InitProducerId">InitProducerId API (Key: 22):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">InitProducerId Request (Version: 0) => transactional_id transaction_timeout_ms
transactional_id => NULLABLE_STRING
transaction_timeout_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The transactional id, or null if the producer is not transactional.</td></tr>
<tr>
<td>transaction_timeout_ms</td><td>The time in ms to wait for before aborting idle transactions sent by this producer. This is only relevant if a TransactionalId has been defined.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">InitProducerId Request (Version: 1) => transactional_id transaction_timeout_ms
transactional_id => NULLABLE_STRING
transaction_timeout_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The transactional id, or null if the producer is not transactional.</td></tr>
<tr>
<td>transaction_timeout_ms</td><td>The time in ms to wait for before aborting idle transactions sent by this producer. This is only relevant if a TransactionalId has been defined.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">InitProducerId Request (Version: 2) => transactional_id transaction_timeout_ms TAG_BUFFER
transactional_id => COMPACT_NULLABLE_STRING
transaction_timeout_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The transactional id, or null if the producer is not transactional.</td></tr>
<tr>
<td>transaction_timeout_ms</td><td>The time in ms to wait for before aborting idle transactions sent by this producer. This is only relevant if a TransactionalId has been defined.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">InitProducerId Response (Version: 0) => throttle_time_ms error_code producer_id producer_epoch
throttle_time_ms => INT32
error_code => INT16
producer_id => INT64
producer_epoch => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>producer_id</td><td>The current producer id.</td></tr>
<tr>
<td>producer_epoch</td><td>The current epoch associated with the producer id.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">InitProducerId Response (Version: 1) => throttle_time_ms error_code producer_id producer_epoch
throttle_time_ms => INT32
error_code => INT16
producer_id => INT64
producer_epoch => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>producer_id</td><td>The current producer id.</td></tr>
<tr>
<td>producer_epoch</td><td>The current epoch associated with the producer id.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">InitProducerId Response (Version: 2) => throttle_time_ms error_code producer_id producer_epoch TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
producer_id => INT64
producer_epoch => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>producer_id</td><td>The current producer id.</td></tr>
<tr>
<td>producer_epoch</td><td>The current epoch associated with the producer id.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_OffsetForLeaderEpoch">OffsetForLeaderEpoch API (Key: 23):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">OffsetForLeaderEpoch Request (Version: 0) => [topics]
topics => topic [partitions]
topic => STRING
partitions => partition leader_epoch
partition => INT32
leader_epoch => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>An array of topics to get epochs for</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>An array of partitions to get epochs for</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>leader_epoch</td><td>The epoch to lookup an offset for.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetForLeaderEpoch Request (Version: 1) => [topics]
topics => topic [partitions]
topic => STRING
partitions => partition leader_epoch
partition => INT32
leader_epoch => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>An array of topics to get epochs for</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>An array of partitions to get epochs for</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>leader_epoch</td><td>The epoch to lookup an offset for.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetForLeaderEpoch Request (Version: 2) => [topics]
topics => topic [partitions]
topic => STRING
partitions => partition current_leader_epoch leader_epoch
partition => INT32
current_leader_epoch => INT32
leader_epoch => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>An array of topics to get epochs for</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>An array of partitions to get epochs for</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>current_leader_epoch</td><td>The current leader epoch, if provided, is used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned.</td></tr>
<tr>
<td>leader_epoch</td><td>The epoch to lookup an offset for.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetForLeaderEpoch Request (Version: 3) => replica_id [topics]
replica_id => INT32
topics => topic [partitions]
topic => STRING
partitions => partition current_leader_epoch leader_epoch
partition => INT32
current_leader_epoch => INT32
leader_epoch => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>replica_id</td><td>Broker id of the follower. For normal consumers, use -1.</td></tr>
<tr>
<td>topics</td><td>An array of topics to get epochs for</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>An array of partitions to get epochs for</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>current_leader_epoch</td><td>The current leader epoch, if provided, is used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned.</td></tr>
<tr>
<td>leader_epoch</td><td>The epoch to lookup an offset for.</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">OffsetForLeaderEpoch Response (Version: 0) => [topics]
topics => topic [partitions]
topic => STRING
partitions => error_code partition end_offset
error_code => INT16
partition => INT32
end_offset => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>An array of topics for which we have leader offsets for some requested partition leader epoch</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>An array of offsets by partition</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>end_offset</td><td>The end offset</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetForLeaderEpoch Response (Version: 1) => [topics]
topics => topic [partitions]
topic => STRING
partitions => error_code partition leader_epoch end_offset
error_code => INT16
partition => INT32
leader_epoch => INT32
end_offset => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>An array of topics for which we have leader offsets for some requested partition leader epoch</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>An array of offsets by partition</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch</td></tr>
<tr>
<td>end_offset</td><td>The end offset</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetForLeaderEpoch Response (Version: 2) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => topic [partitions]
topic => STRING
partitions => error_code partition leader_epoch end_offset
error_code => INT16
partition => INT32
leader_epoch => INT32
end_offset => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>topics</td><td>An array of topics for which we have leader offsets for some requested partition leader epoch</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>An array of offsets by partition</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch</td></tr>
<tr>
<td>end_offset</td><td>The end offset</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">OffsetForLeaderEpoch Response (Version: 3) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => topic [partitions]
topic => STRING
partitions => error_code partition leader_epoch end_offset
error_code => INT16
partition => INT32
leader_epoch => INT32
end_offset => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>topics</td><td>An array of topics for which we have leader offsets for some requested partition leader epoch</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>An array of offsets by partition</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>leader_epoch</td><td>The leader epoch</td></tr>
<tr>
<td>end_offset</td><td>The end offset</td></tr>
</table>
</p>
<h5><a name="The_Messages_AddPartitionsToTxn">AddPartitionsToTxn API (Key: 24):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">AddPartitionsToTxn Request (Version: 0) => transactional_id producer_id producer_epoch [topics]
transactional_id => STRING
producer_id => INT64
producer_epoch => INT16
topics => topic [partitions]
topic => STRING
partitions => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The transactional id corresponding to the transaction.</td></tr>
<tr>
<td>producer_id</td><td>Current producer id in use by the transactional id.</td></tr>
<tr>
<td>producer_epoch</td><td>Current epoch associated with the producer id.</td></tr>
<tr>
<td>topics</td><td>The partitions to add to the transaction.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">AddPartitionsToTxn Request (Version: 1) => transactional_id producer_id producer_epoch [topics]
transactional_id => STRING
producer_id => INT64
producer_epoch => INT16
topics => topic [partitions]
topic => STRING
partitions => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The transactional id corresponding to the transaction.</td></tr>
<tr>
<td>producer_id</td><td>Current producer id in use by the transactional id.</td></tr>
<tr>
<td>producer_epoch</td><td>Current epoch associated with the producer id.</td></tr>
<tr>
<td>topics</td><td>The partitions to add to the transaction.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>null</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">AddPartitionsToTxn Response (Version: 0) => throttle_time_ms [errors]
throttle_time_ms => INT32
errors => topic [partition_errors]
topic => STRING
partition_errors => partition error_code
partition => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>errors</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_errors</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">AddPartitionsToTxn Response (Version: 1) => throttle_time_ms [errors]
throttle_time_ms => INT32
errors => topic [partition_errors]
topic => STRING
partition_errors => partition error_code
partition => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>errors</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partition_errors</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
</table>
</p>
<h5><a name="The_Messages_AddOffsetsToTxn">AddOffsetsToTxn API (Key: 25):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">AddOffsetsToTxn Request (Version: 0) => transactional_id producer_id producer_epoch group_id
transactional_id => STRING
producer_id => INT64
producer_epoch => INT16
group_id => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The transactional id corresponding to the transaction.</td></tr>
<tr>
<td>producer_id</td><td>Current producer id in use by the transactional id.</td></tr>
<tr>
<td>producer_epoch</td><td>Current epoch associated with the producer id.</td></tr>
<tr>
<td>group_id</td><td>The unique group identifier</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">AddOffsetsToTxn Request (Version: 1) => transactional_id producer_id producer_epoch group_id
transactional_id => STRING
producer_id => INT64
producer_epoch => INT16
group_id => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The transactional id corresponding to the transaction.</td></tr>
<tr>
<td>producer_id</td><td>Current producer id in use by the transactional id.</td></tr>
<tr>
<td>producer_epoch</td><td>Current epoch associated with the producer id.</td></tr>
<tr>
<td>group_id</td><td>The unique group identifier</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">AddOffsetsToTxn Response (Version: 0) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">AddOffsetsToTxn Response (Version: 1) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
</table>
</p>
<h5><a name="The_Messages_EndTxn">EndTxn API (Key: 26):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">EndTxn Request (Version: 0) => transactional_id producer_id producer_epoch transaction_result
transactional_id => STRING
producer_id => INT64
producer_epoch => INT16
transaction_result => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The transactional id corresponding to the transaction.</td></tr>
<tr>
<td>producer_id</td><td>Current producer id in use by the transactional id.</td></tr>
<tr>
<td>producer_epoch</td><td>Current epoch associated with the producer id.</td></tr>
<tr>
<td>transaction_result</td><td>The result of the transaction (0 = ABORT, 1 = COMMIT)</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">EndTxn Request (Version: 1) => transactional_id producer_id producer_epoch transaction_result
transactional_id => STRING
producer_id => INT64
producer_epoch => INT16
transaction_result => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The transactional id corresponding to the transaction.</td></tr>
<tr>
<td>producer_id</td><td>Current producer id in use by the transactional id.</td></tr>
<tr>
<td>producer_epoch</td><td>Current epoch associated with the producer id.</td></tr>
<tr>
<td>transaction_result</td><td>The result of the transaction (0 = ABORT, 1 = COMMIT)</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">EndTxn Response (Version: 0) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">EndTxn Response (Version: 1) => throttle_time_ms error_code
throttle_time_ms => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
</table>
</p>
<h5><a name="The_Messages_WriteTxnMarkers">WriteTxnMarkers API (Key: 27):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">WriteTxnMarkers Request (Version: 0) => [transaction_markers]
transaction_markers => producer_id producer_epoch transaction_result [topics] coordinator_epoch
producer_id => INT64
producer_epoch => INT16
transaction_result => BOOLEAN
topics => topic [partitions]
topic => STRING
partitions => INT32
coordinator_epoch => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transaction_markers</td><td>The transaction markers to be written.</td></tr>
<tr>
<td>producer_id</td><td>Current producer id in use by the transactional id.</td></tr>
<tr>
<td>producer_epoch</td><td>Current epoch associated with the producer id.</td></tr>
<tr>
<td>transaction_result</td><td>The result of the transaction to write to the partitions (false = ABORT, true = COMMIT).</td></tr>
<tr>
<td>topics</td><td>The partitions to write markers for.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>null</td></tr>
<tr>
<td>coordinator_epoch</td><td>Epoch associated with the transaction state partition hosted by this transaction coordinator</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">WriteTxnMarkers Response (Version: 0) => [transaction_markers]
transaction_markers => producer_id [topics]
producer_id => INT64
topics => topic [partitions]
topic => STRING
partitions => partition error_code
partition => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transaction_markers</td><td>Errors per partition from writing markers.</td></tr>
<tr>
<td>producer_id</td><td>Current producer id in use by the transactional id.</td></tr>
<tr>
<td>topics</td><td>Errors per partition from writing markers.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
</table>
</p>
<h5><a name="The_Messages_TxnOffsetCommit">TxnOffsetCommit API (Key: 28):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">TxnOffsetCommit Request (Version: 0) => transactional_id group_id producer_id producer_epoch [topics]
transactional_id => STRING
group_id => STRING
producer_id => INT64
producer_epoch => INT16
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset committed_metadata
partition_index => INT32
committed_offset => INT64
committed_metadata => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The ID of the transaction.</td></tr>
<tr>
<td>group_id</td><td>The ID of the group.</td></tr>
<tr>
<td>producer_id</td><td>The current producer ID in use by the transactional ID.</td></tr>
<tr>
<td>producer_epoch</td><td>The current epoch associated with the producer ID.</td></tr>
<tr>
<td>topics</td><td>Each topic that we want to committ offsets for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The partitions inside the topic that we want to committ offsets for.</td></tr>
<tr>
<td>partition_index</td><td>The index of the partition within the topic.</td></tr>
<tr>
<td>committed_offset</td><td>The message offset to be committed.</td></tr>
<tr>
<td>committed_metadata</td><td>Any associated metadata the client wants to keep.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">TxnOffsetCommit Request (Version: 1) => transactional_id group_id producer_id producer_epoch [topics]
transactional_id => STRING
group_id => STRING
producer_id => INT64
producer_epoch => INT16
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset committed_metadata
partition_index => INT32
committed_offset => INT64
committed_metadata => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The ID of the transaction.</td></tr>
<tr>
<td>group_id</td><td>The ID of the group.</td></tr>
<tr>
<td>producer_id</td><td>The current producer ID in use by the transactional ID.</td></tr>
<tr>
<td>producer_epoch</td><td>The current epoch associated with the producer ID.</td></tr>
<tr>
<td>topics</td><td>Each topic that we want to committ offsets for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The partitions inside the topic that we want to committ offsets for.</td></tr>
<tr>
<td>partition_index</td><td>The index of the partition within the topic.</td></tr>
<tr>
<td>committed_offset</td><td>The message offset to be committed.</td></tr>
<tr>
<td>committed_metadata</td><td>Any associated metadata the client wants to keep.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">TxnOffsetCommit Request (Version: 2) => transactional_id group_id producer_id producer_epoch [topics]
transactional_id => STRING
group_id => STRING
producer_id => INT64
producer_epoch => INT16
topics => name [partitions]
name => STRING
partitions => partition_index committed_offset committed_leader_epoch committed_metadata
partition_index => INT32
committed_offset => INT64
committed_leader_epoch => INT32
committed_metadata => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>transactional_id</td><td>The ID of the transaction.</td></tr>
<tr>
<td>group_id</td><td>The ID of the group.</td></tr>
<tr>
<td>producer_id</td><td>The current producer ID in use by the transactional ID.</td></tr>
<tr>
<td>producer_epoch</td><td>The current epoch associated with the producer ID.</td></tr>
<tr>
<td>topics</td><td>Each topic that we want to committ offsets for.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The partitions inside the topic that we want to committ offsets for.</td></tr>
<tr>
<td>partition_index</td><td>The index of the partition within the topic.</td></tr>
<tr>
<td>committed_offset</td><td>The message offset to be committed.</td></tr>
<tr>
<td>committed_leader_epoch</td><td>The leader epoch of the last consumed record.</td></tr>
<tr>
<td>committed_metadata</td><td>Any associated metadata the client wants to keep.</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">TxnOffsetCommit Response (Version: 0) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => name [partitions]
name => STRING
partitions => partition_index error_code
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>The responses for each topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses for each partition in the topic.</td></tr>
<tr>
<td>partition_index</td><td>The partitition index.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">TxnOffsetCommit Response (Version: 1) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => name [partitions]
name => STRING
partitions => partition_index error_code
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>The responses for each topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses for each partition in the topic.</td></tr>
<tr>
<td>partition_index</td><td>The partitition index.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">TxnOffsetCommit Response (Version: 2) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => name [partitions]
name => STRING
partitions => partition_index error_code
partition_index => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>The responses for each topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses for each partition in the topic.</td></tr>
<tr>
<td>partition_index</td><td>The partitition index.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>
<h5><a name="The_Messages_DescribeAcls">DescribeAcls API (Key: 29):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">DescribeAcls Request (Version: 0) => resource_type resource_name principal host operation permission_type
resource_type => INT8
resource_name => NULLABLE_STRING
principal => NULLABLE_STRING
host => NULLABLE_STRING
operation => INT8
permission_type => INT8
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>resource_type</td><td>The resource type</td></tr>
<tr>
<td>resource_name</td><td>The resource name filter</td></tr>
<tr>
<td>principal</td><td>The ACL principal filter</td></tr>
<tr>
<td>host</td><td>The ACL host filter</td></tr>
<tr>
<td>operation</td><td>The ACL operation</td></tr>
<tr>
<td>permission_type</td><td>The ACL permission type</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeAcls Request (Version: 1) => resource_type resource_name resource_pattern_type_filter principal host operation permission_type
resource_type => INT8
resource_name => NULLABLE_STRING
resource_pattern_type_filter => INT8
principal => NULLABLE_STRING
host => NULLABLE_STRING
operation => INT8
permission_type => INT8
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>resource_type</td><td>The resource type</td></tr>
<tr>
<td>resource_name</td><td>The resource name filter</td></tr>
<tr>
<td>resource_pattern_type_filter</td><td>The resource pattern type filter</td></tr>
<tr>
<td>principal</td><td>The ACL principal filter</td></tr>
<tr>
<td>host</td><td>The ACL host filter</td></tr>
<tr>
<td>operation</td><td>The ACL operation</td></tr>
<tr>
<td>permission_type</td><td>The ACL permission type</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">DescribeAcls Response (Version: 0) => throttle_time_ms error_code error_message [resources]
throttle_time_ms => INT32
error_code => INT16
error_message => NULLABLE_STRING
resources => resource_type resource_name [acls]
resource_type => INT8
resource_name => STRING
acls => principal host operation permission_type
principal => STRING
host => STRING
operation => INT8
permission_type => INT8
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>error_message</td><td>Response error message</td></tr>
<tr>
<td>resources</td><td>The resources and their associated ACLs.</td></tr>
<tr>
<td>resource_type</td><td>The resource type</td></tr>
<tr>
<td>resource_name</td><td>The resource name</td></tr>
<tr>
<td>acls</td><td>null</td></tr>
<tr>
<td>principal</td><td>The ACL principal</td></tr>
<tr>
<td>host</td><td>The ACL host</td></tr>
<tr>
<td>operation</td><td>The ACL operation</td></tr>
<tr>
<td>permission_type</td><td>The ACL permission type</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeAcls Response (Version: 1) => throttle_time_ms error_code error_message [resources]
throttle_time_ms => INT32
error_code => INT16
error_message => NULLABLE_STRING
resources => resource_type resource_name resource_pattern_type [acls]
resource_type => INT8
resource_name => STRING
resource_pattern_type => INT8
acls => principal host operation permission_type
principal => STRING
host => STRING
operation => INT8
permission_type => INT8
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>error_message</td><td>Response error message</td></tr>
<tr>
<td>resources</td><td>The resources and their associated ACLs.</td></tr>
<tr>
<td>resource_type</td><td>The resource type</td></tr>
<tr>
<td>resource_name</td><td>The resource name</td></tr>
<tr>
<td>resource_pattern_type</td><td>The resource pattern type</td></tr>
<tr>
<td>acls</td><td>null</td></tr>
<tr>
<td>principal</td><td>The ACL principal</td></tr>
<tr>
<td>host</td><td>The ACL host</td></tr>
<tr>
<td>operation</td><td>The ACL operation</td></tr>
<tr>
<td>permission_type</td><td>The ACL permission type</td></tr>
</table>
</p>
<h5><a name="The_Messages_CreateAcls">CreateAcls API (Key: 30):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">CreateAcls Request (Version: 0) => [creations]
creations => resource_type resource_name principal host operation permission_type
resource_type => INT8
resource_name => STRING
principal => STRING
host => STRING
operation => INT8
permission_type => INT8
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>creations</td><td>null</td></tr>
<tr>
<td>resource_type</td><td>The resource type</td></tr>
<tr>
<td>resource_name</td><td>The resource name</td></tr>
<tr>
<td>principal</td><td>The ACL principal</td></tr>
<tr>
<td>host</td><td>The ACL host</td></tr>
<tr>
<td>operation</td><td>The ACL operation</td></tr>
<tr>
<td>permission_type</td><td>The ACL permission type</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreateAcls Request (Version: 1) => [creations]
creations => resource_type resource_name resource_pattern_type principal host operation permission_type
resource_type => INT8
resource_name => STRING
resource_pattern_type => INT8
principal => STRING
host => STRING
operation => INT8
permission_type => INT8
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>creations</td><td>null</td></tr>
<tr>
<td>resource_type</td><td>The resource type</td></tr>
<tr>
<td>resource_name</td><td>The resource name</td></tr>
<tr>
<td>resource_pattern_type</td><td>The resource pattern type</td></tr>
<tr>
<td>principal</td><td>The ACL principal</td></tr>
<tr>
<td>host</td><td>The ACL host</td></tr>
<tr>
<td>operation</td><td>The ACL operation</td></tr>
<tr>
<td>permission_type</td><td>The ACL permission type</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">CreateAcls Response (Version: 0) => throttle_time_ms [creation_responses]
throttle_time_ms => INT32
creation_responses => error_code error_message
error_code => INT16
error_message => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>creation_responses</td><td>null</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>error_message</td><td>Response error message</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreateAcls Response (Version: 1) => throttle_time_ms [creation_responses]
throttle_time_ms => INT32
creation_responses => error_code error_message
error_code => INT16
error_message => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>creation_responses</td><td>null</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>error_message</td><td>Response error message</td></tr>
</table>
</p>
<h5><a name="The_Messages_DeleteAcls">DeleteAcls API (Key: 31):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">DeleteAcls Request (Version: 0) => [filters]
filters => resource_type resource_name principal host operation permission_type
resource_type => INT8
resource_name => NULLABLE_STRING
principal => NULLABLE_STRING
host => NULLABLE_STRING
operation => INT8
permission_type => INT8
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>filters</td><td>null</td></tr>
<tr>
<td>resource_type</td><td>The resource type</td></tr>
<tr>
<td>resource_name</td><td>The resource name filter</td></tr>
<tr>
<td>principal</td><td>The ACL principal filter</td></tr>
<tr>
<td>host</td><td>The ACL host filter</td></tr>
<tr>
<td>operation</td><td>The ACL operation</td></tr>
<tr>
<td>permission_type</td><td>The ACL permission type</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DeleteAcls Request (Version: 1) => [filters]
filters => resource_type resource_name resource_pattern_type_filter principal host operation permission_type
resource_type => INT8
resource_name => NULLABLE_STRING
resource_pattern_type_filter => INT8
principal => NULLABLE_STRING
host => NULLABLE_STRING
operation => INT8
permission_type => INT8
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>filters</td><td>null</td></tr>
<tr>
<td>resource_type</td><td>The resource type</td></tr>
<tr>
<td>resource_name</td><td>The resource name filter</td></tr>
<tr>
<td>resource_pattern_type_filter</td><td>The resource pattern type filter</td></tr>
<tr>
<td>principal</td><td>The ACL principal filter</td></tr>
<tr>
<td>host</td><td>The ACL host filter</td></tr>
<tr>
<td>operation</td><td>The ACL operation</td></tr>
<tr>
<td>permission_type</td><td>The ACL permission type</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">DeleteAcls Response (Version: 0) => throttle_time_ms [filter_responses]
throttle_time_ms => INT32
filter_responses => error_code error_message [matching_acls]
error_code => INT16
error_message => NULLABLE_STRING
matching_acls => error_code error_message resource_type resource_name principal host operation permission_type
error_code => INT16
error_message => NULLABLE_STRING
resource_type => INT8
resource_name => STRING
principal => STRING
host => STRING
operation => INT8
permission_type => INT8
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>filter_responses</td><td>null</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>error_message</td><td>Response error message</td></tr>
<tr>
<td>matching_acls</td><td>The matching ACLs</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>error_message</td><td>Response error message</td></tr>
<tr>
<td>resource_type</td><td>The resource type</td></tr>
<tr>
<td>resource_name</td><td>The resource name</td></tr>
<tr>
<td>principal</td><td>The ACL principal</td></tr>
<tr>
<td>host</td><td>The ACL host</td></tr>
<tr>
<td>operation</td><td>The ACL operation</td></tr>
<tr>
<td>permission_type</td><td>The ACL permission type</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DeleteAcls Response (Version: 1) => throttle_time_ms [filter_responses]
throttle_time_ms => INT32
filter_responses => error_code error_message [matching_acls]
error_code => INT16
error_message => NULLABLE_STRING
matching_acls => error_code error_message resource_type resource_name resource_pattern_type principal host operation permission_type
error_code => INT16
error_message => NULLABLE_STRING
resource_type => INT8
resource_name => STRING
resource_pattern_type => INT8
principal => STRING
host => STRING
operation => INT8
permission_type => INT8
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>filter_responses</td><td>null</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>error_message</td><td>Response error message</td></tr>
<tr>
<td>matching_acls</td><td>The matching ACLs</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>error_message</td><td>Response error message</td></tr>
<tr>
<td>resource_type</td><td>The resource type</td></tr>
<tr>
<td>resource_name</td><td>The resource name</td></tr>
<tr>
<td>resource_pattern_type</td><td>The resource pattern type</td></tr>
<tr>
<td>principal</td><td>The ACL principal</td></tr>
<tr>
<td>host</td><td>The ACL host</td></tr>
<tr>
<td>operation</td><td>The ACL operation</td></tr>
<tr>
<td>permission_type</td><td>The ACL permission type</td></tr>
</table>
</p>
<h5><a name="The_Messages_DescribeConfigs">DescribeConfigs API (Key: 32):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">DescribeConfigs Request (Version: 0) => [resources]
resources => resource_type resource_name [config_names]
resource_type => INT8
resource_name => STRING
config_names => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>resources</td><td>An array of config resources to be returned.</td></tr>
<tr>
<td>resource_type</td><td>null</td></tr>
<tr>
<td>resource_name</td><td>null</td></tr>
<tr>
<td>config_names</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeConfigs Request (Version: 1) => [resources] include_synonyms
resources => resource_type resource_name [config_names]
resource_type => INT8
resource_name => STRING
config_names => STRING
include_synonyms => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>resources</td><td>An array of config resources to be returned.</td></tr>
<tr>
<td>resource_type</td><td>null</td></tr>
<tr>
<td>resource_name</td><td>null</td></tr>
<tr>
<td>config_names</td><td>null</td></tr>
<tr>
<td>include_synonyms</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeConfigs Request (Version: 2) => [resources] include_synonyms
resources => resource_type resource_name [config_names]
resource_type => INT8
resource_name => STRING
config_names => STRING
include_synonyms => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>resources</td><td>An array of config resources to be returned.</td></tr>
<tr>
<td>resource_type</td><td>null</td></tr>
<tr>
<td>resource_name</td><td>null</td></tr>
<tr>
<td>config_names</td><td>null</td></tr>
<tr>
<td>include_synonyms</td><td>null</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">DescribeConfigs Response (Version: 0) => throttle_time_ms [resources]
throttle_time_ms => INT32
resources => error_code error_message resource_type resource_name [config_entries]
error_code => INT16
error_message => NULLABLE_STRING
resource_type => INT8
resource_name => STRING
config_entries => config_name config_value read_only is_default is_sensitive
config_name => STRING
config_value => NULLABLE_STRING
read_only => BOOLEAN
is_default => BOOLEAN
is_sensitive => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>resources</td><td>null</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>error_message</td><td>Response error message</td></tr>
<tr>
<td>resource_type</td><td>null</td></tr>
<tr>
<td>resource_name</td><td>null</td></tr>
<tr>
<td>config_entries</td><td>null</td></tr>
<tr>
<td>config_name</td><td>null</td></tr>
<tr>
<td>config_value</td><td>null</td></tr>
<tr>
<td>read_only</td><td>null</td></tr>
<tr>
<td>is_default</td><td>null</td></tr>
<tr>
<td>is_sensitive</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeConfigs Response (Version: 1) => throttle_time_ms [resources]
throttle_time_ms => INT32
resources => error_code error_message resource_type resource_name [config_entries]
error_code => INT16
error_message => NULLABLE_STRING
resource_type => INT8
resource_name => STRING
config_entries => config_name config_value read_only config_source is_sensitive [config_synonyms]
config_name => STRING
config_value => NULLABLE_STRING
read_only => BOOLEAN
config_source => INT8
is_sensitive => BOOLEAN
config_synonyms => config_name config_value config_source
config_name => STRING
config_value => NULLABLE_STRING
config_source => INT8
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>resources</td><td>null</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>error_message</td><td>Response error message</td></tr>
<tr>
<td>resource_type</td><td>null</td></tr>
<tr>
<td>resource_name</td><td>null</td></tr>
<tr>
<td>config_entries</td><td>null</td></tr>
<tr>
<td>config_name</td><td>null</td></tr>
<tr>
<td>config_value</td><td>null</td></tr>
<tr>
<td>read_only</td><td>null</td></tr>
<tr>
<td>config_source</td><td>null</td></tr>
<tr>
<td>is_sensitive</td><td>null</td></tr>
<tr>
<td>config_synonyms</td><td>null</td></tr>
<tr>
<td>config_name</td><td>null</td></tr>
<tr>
<td>config_value</td><td>null</td></tr>
<tr>
<td>config_source</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeConfigs Response (Version: 2) => throttle_time_ms [resources]
throttle_time_ms => INT32
resources => error_code error_message resource_type resource_name [config_entries]
error_code => INT16
error_message => NULLABLE_STRING
resource_type => INT8
resource_name => STRING
config_entries => config_name config_value read_only config_source is_sensitive [config_synonyms]
config_name => STRING
config_value => NULLABLE_STRING
read_only => BOOLEAN
config_source => INT8
is_sensitive => BOOLEAN
config_synonyms => config_name config_value config_source
config_name => STRING
config_value => NULLABLE_STRING
config_source => INT8
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>resources</td><td>null</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>error_message</td><td>Response error message</td></tr>
<tr>
<td>resource_type</td><td>null</td></tr>
<tr>
<td>resource_name</td><td>null</td></tr>
<tr>
<td>config_entries</td><td>null</td></tr>
<tr>
<td>config_name</td><td>null</td></tr>
<tr>
<td>config_value</td><td>null</td></tr>
<tr>
<td>read_only</td><td>null</td></tr>
<tr>
<td>config_source</td><td>null</td></tr>
<tr>
<td>is_sensitive</td><td>null</td></tr>
<tr>
<td>config_synonyms</td><td>null</td></tr>
<tr>
<td>config_name</td><td>null</td></tr>
<tr>
<td>config_value</td><td>null</td></tr>
<tr>
<td>config_source</td><td>null</td></tr>
</table>
</p>
<h5><a name="The_Messages_AlterConfigs">AlterConfigs API (Key: 33):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">AlterConfigs Request (Version: 0) => [resources] validate_only
resources => resource_type resource_name [config_entries]
resource_type => INT8
resource_name => STRING
config_entries => config_name config_value
config_name => STRING
config_value => NULLABLE_STRING
validate_only => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>resources</td><td>An array of resources to update with the provided configs.</td></tr>
<tr>
<td>resource_type</td><td>null</td></tr>
<tr>
<td>resource_name</td><td>null</td></tr>
<tr>
<td>config_entries</td><td>null</td></tr>
<tr>
<td>config_name</td><td>Configuration name</td></tr>
<tr>
<td>config_value</td><td>Configuration value</td></tr>
<tr>
<td>validate_only</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">AlterConfigs Request (Version: 1) => [resources] validate_only
resources => resource_type resource_name [config_entries]
resource_type => INT8
resource_name => STRING
config_entries => config_name config_value
config_name => STRING
config_value => NULLABLE_STRING
validate_only => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>resources</td><td>An array of resources to update with the provided configs.</td></tr>
<tr>
<td>resource_type</td><td>null</td></tr>
<tr>
<td>resource_name</td><td>null</td></tr>
<tr>
<td>config_entries</td><td>null</td></tr>
<tr>
<td>config_name</td><td>Configuration name</td></tr>
<tr>
<td>config_value</td><td>Configuration value</td></tr>
<tr>
<td>validate_only</td><td>null</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">AlterConfigs Response (Version: 0) => throttle_time_ms [resources]
throttle_time_ms => INT32
resources => error_code error_message resource_type resource_name
error_code => INT16
error_message => NULLABLE_STRING
resource_type => INT8
resource_name => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>resources</td><td>null</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>error_message</td><td>Response error message</td></tr>
<tr>
<td>resource_type</td><td>null</td></tr>
<tr>
<td>resource_name</td><td>null</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">AlterConfigs Response (Version: 1) => throttle_time_ms [resources]
throttle_time_ms => INT32
resources => error_code error_message resource_type resource_name
error_code => INT16
error_message => NULLABLE_STRING
resource_type => INT8
resource_name => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>resources</td><td>null</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>error_message</td><td>Response error message</td></tr>
<tr>
<td>resource_type</td><td>null</td></tr>
<tr>
<td>resource_name</td><td>null</td></tr>
</table>
</p>
<h5><a name="The_Messages_AlterReplicaLogDirs">AlterReplicaLogDirs API (Key: 34):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">AlterReplicaLogDirs Request (Version: 0) => [log_dirs]
log_dirs => log_dir [topics]
log_dir => STRING
topics => topic [partitions]
topic => STRING
partitions => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>log_dirs</td><td>null</td></tr>
<tr>
<td>log_dir</td><td>The absolute log directory path.</td></tr>
<tr>
<td>topics</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>List of partition ids of the topic.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">AlterReplicaLogDirs Request (Version: 1) => [log_dirs]
log_dirs => log_dir [topics]
log_dir => STRING
topics => topic [partitions]
topic => STRING
partitions => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>log_dirs</td><td>null</td></tr>
<tr>
<td>log_dir</td><td>The absolute log directory path.</td></tr>
<tr>
<td>topics</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>List of partition ids of the topic.</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">AlterReplicaLogDirs Response (Version: 0) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => topic [partitions]
topic => STRING
partitions => partition error_code
partition => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>topics</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">AlterReplicaLogDirs Response (Version: 1) => throttle_time_ms [topics]
throttle_time_ms => INT32
topics => topic [partitions]
topic => STRING
partitions => partition error_code
partition => INT32
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>topics</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
</table>
</p>
<h5><a name="The_Messages_DescribeLogDirs">DescribeLogDirs API (Key: 35):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">DescribeLogDirs Request (Version: 0) => [topics]
topics => topic [partitions]
topic => STRING
partitions => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>List of partition ids of the topic.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeLogDirs Request (Version: 1) => [topics]
topics => topic [partitions]
topic => STRING
partitions => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topics</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>List of partition ids of the topic.</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">DescribeLogDirs Response (Version: 0) => throttle_time_ms [log_dirs]
throttle_time_ms => INT32
log_dirs => error_code log_dir [topics]
error_code => INT16
log_dir => STRING
topics => topic [partitions]
topic => STRING
partitions => partition size offset_lag is_future
partition => INT32
size => INT64
offset_lag => INT64
is_future => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>log_dirs</td><td>null</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>log_dir</td><td>The absolute log directory path.</td></tr>
<tr>
<td>topics</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>size</td><td>The size of the log segments of the partition in bytes.</td></tr>
<tr>
<td>offset_lag</td><td>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)</td></tr>
<tr>
<td>is_future</td><td>True if this log is created by AlterReplicaLogDirsRequest and will replace the current log of the replica in the future.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeLogDirs Response (Version: 1) => throttle_time_ms [log_dirs]
throttle_time_ms => INT32
log_dirs => error_code log_dir [topics]
error_code => INT16
log_dir => STRING
topics => topic [partitions]
topic => STRING
partitions => partition size offset_lag is_future
partition => INT32
size => INT64
offset_lag => INT64
is_future => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>log_dirs</td><td>null</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>log_dir</td><td>The absolute log directory path.</td></tr>
<tr>
<td>topics</td><td>null</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>partitions</td><td>null</td></tr>
<tr>
<td>partition</td><td>Topic partition id</td></tr>
<tr>
<td>size</td><td>The size of the log segments of the partition in bytes.</td></tr>
<tr>
<td>offset_lag</td><td>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)</td></tr>
<tr>
<td>is_future</td><td>True if this log is created by AlterReplicaLogDirsRequest and will replace the current log of the replica in the future.</td></tr>
</table>
</p>
<h5><a name="The_Messages_SaslAuthenticate">SaslAuthenticate API (Key: 36):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">SaslAuthenticate Request (Version: 0) => auth_bytes
auth_bytes => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>auth_bytes</td><td>The SASL authentication bytes from the client, as defined by the SASL mechanism.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">SaslAuthenticate Request (Version: 1) => auth_bytes
auth_bytes => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>auth_bytes</td><td>The SASL authentication bytes from the client, as defined by the SASL mechanism.</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">SaslAuthenticate Response (Version: 0) => error_code error_message auth_bytes
error_code => INT16
error_message => NULLABLE_STRING
auth_bytes => BYTES
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>error_message</td><td>The error message, or null if there was no error.</td></tr>
<tr>
<td>auth_bytes</td><td>The SASL authentication bytes from the server, as defined by the SASL mechanism.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">SaslAuthenticate Response (Version: 1) => error_code error_message auth_bytes session_lifetime_ms
error_code => INT16
error_message => NULLABLE_STRING
auth_bytes => BYTES
session_lifetime_ms => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>error_message</td><td>The error message, or null if there was no error.</td></tr>
<tr>
<td>auth_bytes</td><td>The SASL authentication bytes from the server, as defined by the SASL mechanism.</td></tr>
<tr>
<td>session_lifetime_ms</td><td>The SASL authentication bytes from the server, as defined by the SASL mechanism.</td></tr>
</table>
</p>
<h5><a name="The_Messages_CreatePartitions">CreatePartitions API (Key: 37):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">CreatePartitions Request (Version: 0) => [topic_partitions] timeout validate_only
topic_partitions => topic new_partitions
topic => STRING
new_partitions => count [assignment]
count => INT32
assignment => ARRAY(INT32)
timeout => INT32
validate_only => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topic_partitions</td><td>List of topic and the corresponding new partitions.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>new_partitions</td><td>null</td></tr>
<tr>
<td>count</td><td>The new partition count.</td></tr>
<tr>
<td>assignment</td><td>The assigned brokers.</td></tr>
<tr>
<td>timeout</td><td>The time in ms to wait for the partitions to be created.</td></tr>
<tr>
<td>validate_only</td><td>If true then validate the request, but don't actually increase the number of partitions.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreatePartitions Request (Version: 1) => [topic_partitions] timeout validate_only
topic_partitions => topic new_partitions
topic => STRING
new_partitions => count [assignment]
count => INT32
assignment => ARRAY(INT32)
timeout => INT32
validate_only => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topic_partitions</td><td>List of topic and the corresponding new partitions.</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>new_partitions</td><td>null</td></tr>
<tr>
<td>count</td><td>The new partition count.</td></tr>
<tr>
<td>assignment</td><td>The assigned brokers.</td></tr>
<tr>
<td>timeout</td><td>The time in ms to wait for the partitions to be created.</td></tr>
<tr>
<td>validate_only</td><td>If true then validate the request, but don't actually increase the number of partitions.</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">CreatePartitions Response (Version: 0) => throttle_time_ms [topic_errors]
throttle_time_ms => INT32
topic_errors => topic error_code error_message
topic => STRING
error_code => INT16
error_message => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>topic_errors</td><td>Per topic results for the create partitions request</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>error_message</td><td>Response error message</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreatePartitions Response (Version: 1) => throttle_time_ms [topic_errors]
throttle_time_ms => INT32
topic_errors => topic error_code error_message
topic => STRING
error_code => INT16
error_message => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to quota violation (Zero if the request did not violate any quota)</td></tr>
<tr>
<td>topic_errors</td><td>Per topic results for the create partitions request</td></tr>
<tr>
<td>topic</td><td>Name of topic</td></tr>
<tr>
<td>error_code</td><td>Response error code</td></tr>
<tr>
<td>error_message</td><td>Response error message</td></tr>
</table>
</p>
<h5><a name="The_Messages_CreateDelegationToken">CreateDelegationToken API (Key: 38):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">CreateDelegationToken Request (Version: 0) => [renewers] max_lifetime_ms
renewers => principal_type principal_name
principal_type => STRING
principal_name => STRING
max_lifetime_ms => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>renewers</td><td>A list of those who are allowed to renew this token before it expires.</td></tr>
<tr>
<td>principal_type</td><td>The type of the Kafka principal.</td></tr>
<tr>
<td>principal_name</td><td>The name of the Kafka principal.</td></tr>
<tr>
<td>max_lifetime_ms</td><td>The maximum lifetime of the token in milliseconds, or -1 to use the server side default.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreateDelegationToken Request (Version: 1) => [renewers] max_lifetime_ms
renewers => principal_type principal_name
principal_type => STRING
principal_name => STRING
max_lifetime_ms => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>renewers</td><td>A list of those who are allowed to renew this token before it expires.</td></tr>
<tr>
<td>principal_type</td><td>The type of the Kafka principal.</td></tr>
<tr>
<td>principal_name</td><td>The name of the Kafka principal.</td></tr>
<tr>
<td>max_lifetime_ms</td><td>The maximum lifetime of the token in milliseconds, or -1 to use the server side default.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreateDelegationToken Request (Version: 2) => [renewers] max_lifetime_ms TAG_BUFFER
renewers => principal_type principal_name TAG_BUFFER
principal_type => COMPACT_STRING
principal_name => COMPACT_STRING
max_lifetime_ms => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>renewers</td><td>A list of those who are allowed to renew this token before it expires.</td></tr>
<tr>
<td>principal_type</td><td>The type of the Kafka principal.</td></tr>
<tr>
<td>principal_name</td><td>The name of the Kafka principal.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>max_lifetime_ms</td><td>The maximum lifetime of the token in milliseconds, or -1 to use the server side default.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">CreateDelegationToken Response (Version: 0) => error_code principal_type principal_name issue_timestamp_ms expiry_timestamp_ms max_timestamp_ms token_id hmac throttle_time_ms
error_code => INT16
principal_type => STRING
principal_name => STRING
issue_timestamp_ms => INT64
expiry_timestamp_ms => INT64
max_timestamp_ms => INT64
token_id => STRING
hmac => BYTES
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The top-level error, or zero if there was no error.</td></tr>
<tr>
<td>principal_type</td><td>The principal type of the token owner.</td></tr>
<tr>
<td>principal_name</td><td>The name of the token owner.</td></tr>
<tr>
<td>issue_timestamp_ms</td><td>When this token was generated.</td></tr>
<tr>
<td>expiry_timestamp_ms</td><td>When this token expires.</td></tr>
<tr>
<td>max_timestamp_ms</td><td>The maximum lifetime of this token.</td></tr>
<tr>
<td>token_id</td><td>The token UUID.</td></tr>
<tr>
<td>hmac</td><td>HMAC of the delegation token.</td></tr>
<tr>
<td>throttle_time_ms</td><td>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.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreateDelegationToken Response (Version: 1) => error_code principal_type principal_name issue_timestamp_ms expiry_timestamp_ms max_timestamp_ms token_id hmac throttle_time_ms
error_code => INT16
principal_type => STRING
principal_name => STRING
issue_timestamp_ms => INT64
expiry_timestamp_ms => INT64
max_timestamp_ms => INT64
token_id => STRING
hmac => BYTES
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The top-level error, or zero if there was no error.</td></tr>
<tr>
<td>principal_type</td><td>The principal type of the token owner.</td></tr>
<tr>
<td>principal_name</td><td>The name of the token owner.</td></tr>
<tr>
<td>issue_timestamp_ms</td><td>When this token was generated.</td></tr>
<tr>
<td>expiry_timestamp_ms</td><td>When this token expires.</td></tr>
<tr>
<td>max_timestamp_ms</td><td>The maximum lifetime of this token.</td></tr>
<tr>
<td>token_id</td><td>The token UUID.</td></tr>
<tr>
<td>hmac</td><td>HMAC of the delegation token.</td></tr>
<tr>
<td>throttle_time_ms</td><td>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.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">CreateDelegationToken Response (Version: 2) => error_code principal_type principal_name issue_timestamp_ms expiry_timestamp_ms max_timestamp_ms token_id hmac throttle_time_ms TAG_BUFFER
error_code => INT16
principal_type => COMPACT_STRING
principal_name => COMPACT_STRING
issue_timestamp_ms => INT64
expiry_timestamp_ms => INT64
max_timestamp_ms => INT64
token_id => COMPACT_STRING
hmac => COMPACT_BYTES
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The top-level error, or zero if there was no error.</td></tr>
<tr>
<td>principal_type</td><td>The principal type of the token owner.</td></tr>
<tr>
<td>principal_name</td><td>The name of the token owner.</td></tr>
<tr>
<td>issue_timestamp_ms</td><td>When this token was generated.</td></tr>
<tr>
<td>expiry_timestamp_ms</td><td>When this token expires.</td></tr>
<tr>
<td>max_timestamp_ms</td><td>The maximum lifetime of this token.</td></tr>
<tr>
<td>token_id</td><td>The token UUID.</td></tr>
<tr>
<td>hmac</td><td>HMAC of the delegation token.</td></tr>
<tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_RenewDelegationToken">RenewDelegationToken API (Key: 39):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">RenewDelegationToken Request (Version: 0) => hmac renew_period_ms
hmac => BYTES
renew_period_ms => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>hmac</td><td>The HMAC of the delegation token to be renewed.</td></tr>
<tr>
<td>renew_period_ms</td><td>The renewal time period in milliseconds.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">RenewDelegationToken Request (Version: 1) => hmac renew_period_ms
hmac => BYTES
renew_period_ms => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>hmac</td><td>The HMAC of the delegation token to be renewed.</td></tr>
<tr>
<td>renew_period_ms</td><td>The renewal time period in milliseconds.</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">RenewDelegationToken Response (Version: 0) => error_code expiry_timestamp_ms throttle_time_ms
error_code => INT16
expiry_timestamp_ms => INT64
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>expiry_timestamp_ms</td><td>The timestamp in milliseconds at which this token expires.</td></tr>
<tr>
<td>throttle_time_ms</td><td>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.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">RenewDelegationToken Response (Version: 1) => error_code expiry_timestamp_ms throttle_time_ms
error_code => INT16
expiry_timestamp_ms => INT64
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>expiry_timestamp_ms</td><td>The timestamp in milliseconds at which this token expires.</td></tr>
<tr>
<td>throttle_time_ms</td><td>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.</td></tr>
</table>
</p>
<h5><a name="The_Messages_ExpireDelegationToken">ExpireDelegationToken API (Key: 40):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">ExpireDelegationToken Request (Version: 0) => hmac expiry_time_period_ms
hmac => BYTES
expiry_time_period_ms => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>hmac</td><td>The HMAC of the delegation token to be expired.</td></tr>
<tr>
<td>expiry_time_period_ms</td><td>The expiry time period in milliseconds.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ExpireDelegationToken Request (Version: 1) => hmac expiry_time_period_ms
hmac => BYTES
expiry_time_period_ms => INT64
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>hmac</td><td>The HMAC of the delegation token to be expired.</td></tr>
<tr>
<td>expiry_time_period_ms</td><td>The expiry time period in milliseconds.</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">ExpireDelegationToken Response (Version: 0) => error_code expiry_timestamp_ms throttle_time_ms
error_code => INT16
expiry_timestamp_ms => INT64
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>expiry_timestamp_ms</td><td>The timestamp in milliseconds at which this token expires.</td></tr>
<tr>
<td>throttle_time_ms</td><td>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.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ExpireDelegationToken Response (Version: 1) => error_code expiry_timestamp_ms throttle_time_ms
error_code => INT16
expiry_timestamp_ms => INT64
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>expiry_timestamp_ms</td><td>The timestamp in milliseconds at which this token expires.</td></tr>
<tr>
<td>throttle_time_ms</td><td>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.</td></tr>
</table>
</p>
<h5><a name="The_Messages_DescribeDelegationToken">DescribeDelegationToken API (Key: 41):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">DescribeDelegationToken Request (Version: 0) => [owners]
owners => principal_type principal_name
principal_type => STRING
principal_name => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>owners</td><td>Each owner that we want to describe delegation tokens for, or null to describe all tokens.</td></tr>
<tr>
<td>principal_type</td><td>The owner principal type.</td></tr>
<tr>
<td>principal_name</td><td>The owner principal name.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeDelegationToken Request (Version: 1) => [owners]
owners => principal_type principal_name
principal_type => STRING
principal_name => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>owners</td><td>Each owner that we want to describe delegation tokens for, or null to describe all tokens.</td></tr>
<tr>
<td>principal_type</td><td>The owner principal type.</td></tr>
<tr>
<td>principal_name</td><td>The owner principal name.</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">DescribeDelegationToken Response (Version: 0) => error_code [tokens] throttle_time_ms
error_code => INT16
tokens => principal_type principal_name issue_timestamp expiry_timestamp max_timestamp token_id hmac [renewers]
principal_type => STRING
principal_name => STRING
issue_timestamp => INT64
expiry_timestamp => INT64
max_timestamp => INT64
token_id => STRING
hmac => BYTES
renewers => principal_type principal_name
principal_type => STRING
principal_name => STRING
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>tokens</td><td>The tokens.</td></tr>
<tr>
<td>principal_type</td><td>The token principal type.</td></tr>
<tr>
<td>principal_name</td><td>The token principal name.</td></tr>
<tr>
<td>issue_timestamp</td><td>The token issue timestamp in milliseconds.</td></tr>
<tr>
<td>expiry_timestamp</td><td>The token expiry timestamp in milliseconds.</td></tr>
<tr>
<td>max_timestamp</td><td>The token maximum timestamp length in milliseconds.</td></tr>
<tr>
<td>token_id</td><td>The token ID.</td></tr>
<tr>
<td>hmac</td><td>The token HMAC.</td></tr>
<tr>
<td>renewers</td><td>Those who are able to renew this token before it expires.</td></tr>
<tr>
<td>principal_type</td><td>The renewer principal type</td></tr>
<tr>
<td>principal_name</td><td>The renewer principal name</td></tr>
<tr>
<td>throttle_time_ms</td><td>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.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DescribeDelegationToken Response (Version: 1) => error_code [tokens] throttle_time_ms
error_code => INT16
tokens => principal_type principal_name issue_timestamp expiry_timestamp max_timestamp token_id hmac [renewers]
principal_type => STRING
principal_name => STRING
issue_timestamp => INT64
expiry_timestamp => INT64
max_timestamp => INT64
token_id => STRING
hmac => BYTES
renewers => principal_type principal_name
principal_type => STRING
principal_name => STRING
throttle_time_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
<tr>
<td>tokens</td><td>The tokens.</td></tr>
<tr>
<td>principal_type</td><td>The token principal type.</td></tr>
<tr>
<td>principal_name</td><td>The token principal name.</td></tr>
<tr>
<td>issue_timestamp</td><td>The token issue timestamp in milliseconds.</td></tr>
<tr>
<td>expiry_timestamp</td><td>The token expiry timestamp in milliseconds.</td></tr>
<tr>
<td>max_timestamp</td><td>The token maximum timestamp length in milliseconds.</td></tr>
<tr>
<td>token_id</td><td>The token ID.</td></tr>
<tr>
<td>hmac</td><td>The token HMAC.</td></tr>
<tr>
<td>renewers</td><td>Those who are able to renew this token before it expires.</td></tr>
<tr>
<td>principal_type</td><td>The renewer principal type</td></tr>
<tr>
<td>principal_name</td><td>The renewer principal name</td></tr>
<tr>
<td>throttle_time_ms</td><td>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.</td></tr>
</table>
</p>
<h5><a name="The_Messages_DeleteGroups">DeleteGroups API (Key: 42):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">DeleteGroups Request (Version: 0) => [groups_names]
groups_names => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>groups_names</td><td>The group names to delete.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DeleteGroups Request (Version: 1) => [groups_names]
groups_names => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>groups_names</td><td>The group names to delete.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DeleteGroups Request (Version: 2) => [groups_names] TAG_BUFFER
groups_names => COMPACT_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>groups_names</td><td>The group names to delete.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">DeleteGroups Response (Version: 0) => throttle_time_ms [results]
throttle_time_ms => INT32
results => group_id error_code
group_id => STRING
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>results</td><td>The deletion results</td></tr>
<tr>
<td>group_id</td><td>The group id</td></tr>
<tr>
<td>error_code</td><td>The deletion error, or 0 if the deletion succeeded.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DeleteGroups Response (Version: 1) => throttle_time_ms [results]
throttle_time_ms => INT32
results => group_id error_code
group_id => STRING
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>results</td><td>The deletion results</td></tr>
<tr>
<td>group_id</td><td>The group id</td></tr>
<tr>
<td>error_code</td><td>The deletion error, or 0 if the deletion succeeded.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">DeleteGroups Response (Version: 2) => throttle_time_ms [results] TAG_BUFFER
throttle_time_ms => INT32
results => group_id error_code TAG_BUFFER
group_id => COMPACT_STRING
error_code => INT16
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>results</td><td>The deletion results</td></tr>
<tr>
<td>group_id</td><td>The group id</td></tr>
<tr>
<td>error_code</td><td>The deletion error, or 0 if the deletion succeeded.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_ElectLeaders">ElectLeaders API (Key: 43):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">ElectLeaders Request (Version: 0) => [topic_partitions] timeout_ms
topic_partitions => topic [partition_id]
topic => STRING
partition_id => INT32
timeout_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>topic_partitions</td><td>The topic partitions to elect leaders.</td></tr>
<tr>
<td>topic</td><td>The name of a topic.</td></tr>
<tr>
<td>partition_id</td><td>The partitions of this topic whose leader should be elected.</td></tr>
<tr>
<td>timeout_ms</td><td>The time in ms to wait for the election to complete.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ElectLeaders Request (Version: 1) => election_type [topic_partitions] timeout_ms
election_type => INT8
topic_partitions => topic [partition_id]
topic => STRING
partition_id => INT32
timeout_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>election_type</td><td>Type of elections to conduct for the partition. A value of '0' elects the preferred replica. A value of '1' elects the first live replica if there are no in-sync replica.</td></tr>
<tr>
<td>topic_partitions</td><td>The topic partitions to elect leaders.</td></tr>
<tr>
<td>topic</td><td>The name of a topic.</td></tr>
<tr>
<td>partition_id</td><td>The partitions of this topic whose leader should be elected.</td></tr>
<tr>
<td>timeout_ms</td><td>The time in ms to wait for the election to complete.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ElectLeaders Request (Version: 2) => election_type [topic_partitions] timeout_ms TAG_BUFFER
election_type => INT8
topic_partitions => topic [partition_id] TAG_BUFFER
topic => COMPACT_STRING
partition_id => INT32
timeout_ms => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>election_type</td><td>Type of elections to conduct for the partition. A value of '0' elects the preferred replica. A value of '1' elects the first live replica if there are no in-sync replica.</td></tr>
<tr>
<td>topic_partitions</td><td>The topic partitions to elect leaders.</td></tr>
<tr>
<td>topic</td><td>The name of a topic.</td></tr>
<tr>
<td>partition_id</td><td>The partitions of this topic whose leader should be elected.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>timeout_ms</td><td>The time in ms to wait for the election to complete.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">ElectLeaders Response (Version: 0) => throttle_time_ms [replica_election_results]
throttle_time_ms => INT32
replica_election_results => topic [partition_result]
topic => STRING
partition_result => partition_id error_code error_message
partition_id => INT32
error_code => INT16
error_message => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>replica_election_results</td><td>The election results, or an empty array if the requester did not have permission and the request asks for all partitions.</td></tr>
<tr>
<td>topic</td><td>The topic name</td></tr>
<tr>
<td>partition_result</td><td>The results for each partition</td></tr>
<tr>
<td>partition_id</td><td>The partition id</td></tr>
<tr>
<td>error_code</td><td>The result error, or zero if there was no error.</td></tr>
<tr>
<td>error_message</td><td>The result message, or null if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ElectLeaders Response (Version: 1) => throttle_time_ms error_code [replica_election_results]
throttle_time_ms => INT32
error_code => INT16
replica_election_results => topic [partition_result]
topic => STRING
partition_result => partition_id error_code error_message
partition_id => INT32
error_code => INT16
error_message => NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The top level response error code.</td></tr>
<tr>
<td>replica_election_results</td><td>The election results, or an empty array if the requester did not have permission and the request asks for all partitions.</td></tr>
<tr>
<td>topic</td><td>The topic name</td></tr>
<tr>
<td>partition_result</td><td>The results for each partition</td></tr>
<tr>
<td>partition_id</td><td>The partition id</td></tr>
<tr>
<td>error_code</td><td>The result error, or zero if there was no error.</td></tr>
<tr>
<td>error_message</td><td>The result message, or null if there was no error.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">ElectLeaders Response (Version: 2) => throttle_time_ms error_code [replica_election_results] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
replica_election_results => topic [partition_result] TAG_BUFFER
topic => COMPACT_STRING
partition_result => partition_id error_code error_message TAG_BUFFER
partition_id => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The top level response error code.</td></tr>
<tr>
<td>replica_election_results</td><td>The election results, or an empty array if the requester did not have permission and the request asks for all partitions.</td></tr>
<tr>
<td>topic</td><td>The topic name</td></tr>
<tr>
<td>partition_result</td><td>The results for each partition</td></tr>
<tr>
<td>partition_id</td><td>The partition id</td></tr>
<tr>
<td>error_code</td><td>The result error, or zero if there was no error.</td></tr>
<tr>
<td>error_message</td><td>The result message, or null if there was no error.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_IncrementalAlterConfigs">IncrementalAlterConfigs API (Key: 44):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">IncrementalAlterConfigs Request (Version: 0) => [resources] validate_only
resources => resource_type resource_name [configs]
resource_type => INT8
resource_name => STRING
configs => name config_operation value
name => STRING
config_operation => INT8
value => NULLABLE_STRING
validate_only => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>resources</td><td>The incremental updates for each resource.</td></tr>
<tr>
<td>resource_type</td><td>The resource type.</td></tr>
<tr>
<td>resource_name</td><td>The resource name.</td></tr>
<tr>
<td>configs</td><td>The configurations.</td></tr>
<tr>
<td>name</td><td>The configuration key name.</td></tr>
<tr>
<td>config_operation</td><td>The type (Set, Delete, Append, Subtract) of operation.</td></tr>
<tr>
<td>value</td><td>The value to set for the configuration key.</td></tr>
<tr>
<td>validate_only</td><td>True if we should validate the request, but not change the configurations.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">IncrementalAlterConfigs Request (Version: 1) => [resources] validate_only TAG_BUFFER
resources => resource_type resource_name [configs] TAG_BUFFER
resource_type => INT8
resource_name => COMPACT_STRING
configs => name config_operation value TAG_BUFFER
name => COMPACT_STRING
config_operation => INT8
value => COMPACT_NULLABLE_STRING
validate_only => BOOLEAN
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>resources</td><td>The incremental updates for each resource.</td></tr>
<tr>
<td>resource_type</td><td>The resource type.</td></tr>
<tr>
<td>resource_name</td><td>The resource name.</td></tr>
<tr>
<td>configs</td><td>The configurations.</td></tr>
<tr>
<td>name</td><td>The configuration key name.</td></tr>
<tr>
<td>config_operation</td><td>The type (Set, Delete, Append, Subtract) of operation.</td></tr>
<tr>
<td>value</td><td>The value to set for the configuration key.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>validate_only</td><td>True if we should validate the request, but not change the configurations.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">IncrementalAlterConfigs Response (Version: 0) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => error_code error_message resource_type resource_name
error_code => INT16
error_message => NULLABLE_STRING
resource_type => INT8
resource_name => STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.</td></tr>
<tr>
<td>responses</td><td>The responses for each resource.</td></tr>
<tr>
<td>error_code</td><td>The resource error code.</td></tr>
<tr>
<td>error_message</td><td>The resource error message, or null if there was no error.</td></tr>
<tr>
<td>resource_type</td><td>The resource type.</td></tr>
<tr>
<td>resource_name</td><td>The resource name.</td></tr>
</table>
</p>
<p><pre class="line-numbers"><code class="language-java">IncrementalAlterConfigs Response (Version: 1) => throttle_time_ms [responses] TAG_BUFFER
throttle_time_ms => INT32
responses => error_code error_message resource_type resource_name TAG_BUFFER
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
resource_type => INT8
resource_name => COMPACT_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota.</td></tr>
<tr>
<td>responses</td><td>The responses for each resource.</td></tr>
<tr>
<td>error_code</td><td>The resource error code.</td></tr>
<tr>
<td>error_message</td><td>The resource error message, or null if there was no error.</td></tr>
<tr>
<td>resource_type</td><td>The resource type.</td></tr>
<tr>
<td>resource_name</td><td>The resource name.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_AlterPartitionReassignments">AlterPartitionReassignments API (Key: 45):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">AlterPartitionReassignments Request (Version: 0) => timeout_ms [topics] TAG_BUFFER
timeout_ms => INT32
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index [replicas] TAG_BUFFER
partition_index => INT32
replicas => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>timeout_ms</td><td>The time in ms to wait for the request to complete.</td></tr>
<tr>
<td>topics</td><td>The topics to reassign.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The partitions to reassign.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>replicas</td><td>The replicas to place the partitions on, or null to cancel a pending reassignment for this partition.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">AlterPartitionReassignments Response (Version: 0) => throttle_time_ms error_code error_message [responses] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
responses => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index error_code error_message TAG_BUFFER
partition_index => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The top-level error code, or 0 if there was no error.</td></tr>
<tr>
<td>error_message</td><td>The top-level error message, or null if there was no error.</td></tr>
<tr>
<td>responses</td><td>The responses to topics to reassign.</td></tr>
<tr>
<td>name</td><td>The topic name</td></tr>
<tr>
<td>partitions</td><td>The responses to partitions to reassign</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The error code for this partition, or 0 if there was no error.</td></tr>
<tr>
<td>error_message</td><td>The error message for this partition, or null if there was no error.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_ListPartitionReassignments">ListPartitionReassignments API (Key: 46):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">ListPartitionReassignments Request (Version: 0) => timeout_ms [topics] TAG_BUFFER
timeout_ms => INT32
topics => name [partition_indexes] TAG_BUFFER
name => COMPACT_STRING
partition_indexes => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>timeout_ms</td><td>The time in ms to wait for the request to complete.</td></tr>
<tr>
<td>topics</td><td>The topics to list partition reassignments for, or null to list everything.</td></tr>
<tr>
<td>name</td><td>The topic name</td></tr>
<tr>
<td>partition_indexes</td><td>The partitions to list partition reassignments for.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">ListPartitionReassignments Response (Version: 0) => throttle_time_ms error_code error_message [topics] TAG_BUFFER
throttle_time_ms => INT32
error_code => INT16
error_message => COMPACT_NULLABLE_STRING
topics => name [partitions] TAG_BUFFER
name => COMPACT_STRING
partitions => partition_index [replicas] [adding_replicas] [removing_replicas] TAG_BUFFER
partition_index => INT32
replicas => INT32
adding_replicas => INT32
removing_replicas => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>error_code</td><td>The top-level error code, or 0 if there was no error</td></tr>
<tr>
<td>error_message</td><td>The top-level error message, or null if there was no error.</td></tr>
<tr>
<td>topics</td><td>The ongoing reassignments for each topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The ongoing reassignments for each partition.</td></tr>
<tr>
<td>partition_index</td><td>The index of the partition.</td></tr>
<tr>
<td>replicas</td><td>The current replica set.</td></tr>
<tr>
<td>adding_replicas</td><td>The set of replicas we are currently adding.</td></tr>
<tr>
<td>removing_replicas</td><td>The set of replicas we are currently removing.</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
<tr>
<td>_tagged_fields</td><td>The tagged fields</td></tr>
</table>
</p>
<h5><a name="The_Messages_OffsetDelete">OffsetDelete API (Key: 47):</a></h5>
<b>Requests:</b><br>
<p><pre class="line-numbers"><code class="language-java">OffsetDelete Request (Version: 0) => group_id [topics]
group_id => STRING
topics => name [partitions]
name => STRING
partitions => partition_index
partition_index => INT32
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>group_id</td><td>The unique group identifier.</td></tr>
<tr>
<td>topics</td><td>The topics to delete offsets for</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>Each partition to delete offsets for.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
</table>
</p>
<b>Responses:</b><br>
<p><pre class="line-numbers"><code class="language-java">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
</code></pre><table class="data-table"><tbody>
<tr><th>Field</th>
<th>Description</th>
</tr><tr>
<td>error_code</td><td>The top-level error code, or 0 if there was no error.</td></tr>
<tr>
<td>throttle_time_ms</td><td>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.</td></tr>
<tr>
<td>topics</td><td>The responses for each topic.</td></tr>
<tr>
<td>name</td><td>The topic name.</td></tr>
<tr>
<td>partitions</td><td>The responses for each partition in the topic.</td></tr>
<tr>
<td>partition_index</td><td>The partition index.</td></tr>
<tr>
<td>error_code</td><td>The error code, or 0 if there was no error.</td></tr>
</table>
</p>