blob: eb64156e0a337cf55dc8bdb725fc50714128ce06 [file] [log] [blame]
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<script id="ops-template" type="text/x-handlebars-template">
Here is some information on actually running Kafka as a production system based on usage and experience at LinkedIn. Please send us any additional tips you know of.
<h3 class="anchor-heading"><a id="basic_ops" class="anchor-link"></a><a href="#basic_ops">6.1 Basic Kafka Operations</a></h3>
This section will review the most common operations you will perform on your Kafka cluster. All of the tools reviewed in this section are available under the <code>bin/</code> directory of the Kafka distribution and each tool will print details on all possible commandline options if it is run with no arguments.
<h4 class="anchor-heading"><a id="basic_ops_add_topic" class="anchor-link"></a><a href="#basic_ops_add_topic">Adding and removing topics</a></h4>
You have the option of either adding topics manually or having them be created automatically when data is first published to a non-existent topic. If topics are auto-created then you may want to tune the default <a href="#topicconfigs">topic configurations</a> used for auto-created topics.
<p>
Topics are added and modified using the topic tool:
<pre class="line-numbers"><code class="language-bash"> &gt; bin/kafka-topics.sh --bootstrap-server broker_host:port --create --topic my_topic_name \
--partitions 20 --replication-factor 3 --config x=y</code></pre>
The replication factor controls how many servers will replicate each message that is written. If you have a replication factor of 3 then up to 2 servers can fail before you will lose access to your data. We recommend you use a replication factor of 2 or 3 so that you can transparently bounce machines without interrupting data consumption.
<p>
The partition count controls how many logs the topic will be sharded into. There are several impacts of the partition count. First each partition must fit entirely on a single server. So if you have 20 partitions the full data set (and read and write load) will be handled by no more than 20 servers (not counting replicas). Finally the partition count impacts the maximum parallelism of your consumers. This is discussed in greater detail in the <a href="#intro_consumers">concepts section</a>.
<p>
Each sharded partition log is placed into its own folder under the Kafka log directory. The name of such folders consists of the topic name, appended by a dash (-) and the partition id. Since a typical folder name can not be over 255 characters long, there will be a limitation on the length of topic names. We assume the number of partitions will not ever be above 100,000. Therefore, topic names cannot be longer than 249 characters. This leaves just enough room in the folder name for a dash and a potentially 5 digit long partition id.
<p>
The configurations added on the command line override the default settings the server has for things like the length of time data should be retained. The complete set of per-topic configurations is documented <a href="#topicconfigs">here</a>.
<h4 class="anchor-heading"><a id="basic_ops_modify_topic" class="anchor-link"></a><a href="#basic_ops_modify_topic">Modifying topics</a></h4>
You can change the configuration or partitioning of a topic using the same topic tool.
<p>
To add partitions you can do
<pre class="line-numbers"><code class="language-bash"> &gt; bin/kafka-topics.sh --bootstrap-server broker_host:port --alter --topic my_topic_name \
--partitions 40</code></pre>
Be aware that one use case for partitions is to semantically partition data, and adding partitions doesn't change the partitioning of existing data so this may disturb consumers if they rely on that partition. That is if data is partitioned by <code>hash(key) % number_of_partitions</code> then this partitioning will potentially be shuffled by adding partitions but Kafka will not attempt to automatically redistribute data in any way.
<p>
To add configs:
<pre class="line-numbers"><code class="language-bash"> &gt; bin/kafka-configs.sh --bootstrap-server broker_host:port --entity-type topics --entity-name my_topic_name --alter --add-config x=y</code></pre>
To remove a config:
<pre class="line-numbers"><code class="language-bash"> &gt; bin/kafka-configs.sh --bootstrap-server broker_host:port --entity-type topics --entity-name my_topic_name --alter --delete-config x</code></pre>
And finally deleting a topic:
<pre class="line-numbers"><code class="language-bash"> &gt; bin/kafka-topics.sh --bootstrap-server broker_host:port --delete --topic my_topic_name</code></pre>
<p>
Kafka does not currently support reducing the number of partitions for a topic.
<p>
Instructions for changing the replication factor of a topic can be found <a href="#basic_ops_increase_replication_factor">here</a>.
<h4 class="anchor-heading"><a id="basic_ops_restarting" class="anchor-link"></a><a href="#basic_ops_restarting">Graceful shutdown</a></h4>
The Kafka cluster will automatically detect any broker shutdown or failure and elect new leaders for the partitions on that machine. This will occur whether a server fails or it is brought down intentionally for maintenance or configuration changes. For the latter cases Kafka supports a more graceful mechanism for stopping a server than just killing it.
When a server is stopped gracefully it has two optimizations it will take advantage of:
<ol>
<li>It will sync all its logs to disk to avoid needing to do any log recovery when it restarts (i.e. validating the checksum for all messages in the tail of the log). Log recovery takes time so this speeds up intentional restarts.
<li>It will migrate any partitions the server is the leader for to other replicas prior to shutting down. This will make the leadership transfer faster and minimize the time each partition is unavailable to a few milliseconds.
</ol>
Syncing the logs will happen automatically whenever the server is stopped other than by a hard kill, but the controlled leadership migration requires using a special setting:
<pre class="line-numbers"><code class="language-text"> controlled.shutdown.enable=true</code></pre>
Note that controlled shutdown will only succeed if <i>all</i> the partitions hosted on the broker have replicas (i.e. the replication factor is greater than 1 <i>and</i> at least one of these replicas is alive). This is generally what you want since shutting down the last replica would make that topic partition unavailable.
<h4 class="anchor-heading"><a id="basic_ops_leader_balancing" class="anchor-link"></a><a href="#basic_ops_leader_balancing">Balancing leadership</a></h4>
Whenever a broker stops or crashes, leadership for that broker's partitions transfers to other replicas. When the broker is restarted it will only be a follower for all its partitions, meaning it will not be used for client reads and writes.
<p>
To avoid this imbalance, Kafka has a notion of preferred replicas. If the list of replicas for a partition is 1,5,9 then node 1 is preferred as the leader to either node 5 or 9 because it is earlier in the replica list. By default the Kafka cluster will try to restore leadership to the restored replicas. This behaviour is configured with:
<pre class="line-numbers"><code class="language-text"> auto.leader.rebalance.enable=true</code></pre>
You can also set this to false, but you will then need to manually restore leadership to the restored replicas by running the command:
<pre class="line-numbers"><code class="language-bash"> &gt; bin/kafka-preferred-replica-election.sh --bootstrap-server broker_host:port</code></pre>
<h4 class="anchor-heading"><a id="basic_ops_racks" class="anchor-link"></a><a href="#basic_ops_racks">Balancing Replicas Across Racks</a></h4>
The rack awareness feature spreads replicas of the same partition across different racks. This extends the guarantees Kafka provides for broker-failure to cover rack-failure, limiting the risk of data loss should all the brokers on a rack fail at once. The feature can also be applied to other broker groupings such as availability zones in EC2.
<p></p>
You can specify that a broker belongs to a particular rack by adding a property to the broker config:
<pre class="language-text"><code class="language-text"> broker.rack=my-rack-id</code></pre>
When a topic is <a href="#basic_ops_add_topic">created</a>, <a href="#basic_ops_modify_topic">modified</a> or replicas are <a href="#basic_ops_cluster_expansion">redistributed</a>, the rack constraint will be honoured, ensuring replicas span as many racks as they can (a partition will span min(#racks, replication-factor) different racks).
<p></p>
The algorithm used to assign replicas to brokers ensures that the number of leaders per broker will be constant, regardless of how brokers are distributed across racks. This ensures balanced throughput.
<p></p>
However if racks are assigned different numbers of brokers, the assignment of replicas will not be even. Racks with fewer brokers will get more replicas, meaning they will use more storage and put more resources into replication. Hence it is sensible to configure an equal number of brokers per rack.
<h4 class="anchor-heading"><a id="basic_ops_mirror_maker" class="anchor-link"></a><a href="#basic_ops_mirror_maker">Mirroring data between clusters & Geo-replication</a></h4>
<p>
Kafka administrators can define data flows that cross the boundaries of individual Kafka clusters, data centers, or geographical regions. Please refer to the section on <a href="#georeplication">Geo-Replication</a> for further information.
</p>
<h4 class="anchor-heading"><a id="basic_ops_consumer_lag" class="anchor-link"></a><a href="#basic_ops_consumer_lag">Checking consumer position</a></h4>
Sometimes it's useful to see the position of your consumers. We have a tool that will show the position of all consumers in a consumer group as well as how far behind the end of the log they are. To run this tool on a consumer group named <i>my-group</i> consuming a topic named <i>my-topic</i> would look like this:
<pre class="line-numbers"><code class="language-bash"> &gt; bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --describe --group my-group
TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID HOST CLIENT-ID
my-topic 0 2 4 2 consumer-1-029af89c-873c-4751-a720-cefd41a669d6 /127.0.0.1 consumer-1
my-topic 1 2 3 1 consumer-1-029af89c-873c-4751-a720-cefd41a669d6 /127.0.0.1 consumer-1
my-topic 2 2 3 1 consumer-2-42c1abd4-e3b2-425d-a8bb-e1ea49b29bb2 /127.0.0.1 consumer-2</code></pre>
<h4 class="anchor-heading"><a id="basic_ops_consumer_group" class="anchor-link"></a><a href="#basic_ops_consumer_group">Managing Consumer Groups</a></h4>
With the ConsumerGroupCommand tool, we can list, describe, or delete the consumer groups. The consumer group can be deleted manually, or automatically when the last committed offset for that group expires. Manual deletion works only if the group does not have any active members.
For example, to list all consumer groups across all topics:
<pre class="line-numbers"><code class="language-bash"> &gt; bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --list
test-consumer-group</code></pre>
To view offsets, as mentioned earlier, we "describe" the consumer group like this:
<pre class="line-numbers"><code class="language-bash"> &gt; bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --describe --group my-group
TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID HOST CLIENT-ID
topic3 0 241019 395308 154289 consumer2-e76ea8c3-5d30-4299-9005-47eb41f3d3c4 /127.0.0.1 consumer2
topic2 1 520678 803288 282610 consumer2-e76ea8c3-5d30-4299-9005-47eb41f3d3c4 /127.0.0.1 consumer2
topic3 1 241018 398817 157799 consumer2-e76ea8c3-5d30-4299-9005-47eb41f3d3c4 /127.0.0.1 consumer2
topic1 0 854144 855809 1665 consumer1-3fc8d6f1-581a-4472-bdf3-3515b4aee8c1 /127.0.0.1 consumer1
topic2 0 460537 803290 342753 consumer1-3fc8d6f1-581a-4472-bdf3-3515b4aee8c1 /127.0.0.1 consumer1
topic3 2 243655 398812 155157 consumer4-117fe4d3-c6c1-4178-8ee9-eb4a3954bee0 /127.0.0.1 consumer4</code></pre>
There are a number of additional "describe" options that can be used to provide more detailed information about a consumer group:
<ul>
<li>--members: This option provides the list of all active members in the consumer group.
<pre class="line-numbers"><code class="language-bash"> &gt; bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --describe --group my-group --members
CONSUMER-ID HOST CLIENT-ID #PARTITIONS
consumer1-3fc8d6f1-581a-4472-bdf3-3515b4aee8c1 /127.0.0.1 consumer1 2
consumer4-117fe4d3-c6c1-4178-8ee9-eb4a3954bee0 /127.0.0.1 consumer4 1
consumer2-e76ea8c3-5d30-4299-9005-47eb41f3d3c4 /127.0.0.1 consumer2 3
consumer3-ecea43e4-1f01-479f-8349-f9130b75d8ee /127.0.0.1 consumer3 0</code></pre>
</li>
<li>--members --verbose: On top of the information reported by the "--members" options above, this option also provides the partitions assigned to each member.
<pre class="line-numbers"><code class="language-bash"> &gt; bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --describe --group my-group --members --verbose
CONSUMER-ID HOST CLIENT-ID #PARTITIONS ASSIGNMENT
consumer1-3fc8d6f1-581a-4472-bdf3-3515b4aee8c1 /127.0.0.1 consumer1 2 topic1(0), topic2(0)
consumer4-117fe4d3-c6c1-4178-8ee9-eb4a3954bee0 /127.0.0.1 consumer4 1 topic3(2)
consumer2-e76ea8c3-5d30-4299-9005-47eb41f3d3c4 /127.0.0.1 consumer2 3 topic2(1), topic3(0,1)
consumer3-ecea43e4-1f01-479f-8349-f9130b75d8ee /127.0.0.1 consumer3 0 -</code></pre>
</li>
<li>--offsets: This is the default describe option and provides the same output as the "--describe" option.</li>
<li>--state: This option provides useful group-level information.
<pre class="line-numbers"><code class="language-bash"> &gt; bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --describe --group my-group --state
COORDINATOR (ID) ASSIGNMENT-STRATEGY STATE #MEMBERS
localhost:9092 (0) range Stable 4</code></pre>
</li>
</ul>
To manually delete one or multiple consumer groups, the "--delete" option can be used:
<pre class="line-numbers"><code class="language-bash"> &gt; bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --delete --group my-group --group my-other-group
Deletion of requested consumer groups ('my-group', 'my-other-group') was successful.</code></pre>
<p>
To reset offsets of a consumer group, "--reset-offsets" option can be used.
This option supports one consumer group at the time. It requires defining following scopes: --all-topics or --topic. One scope must be selected, unless you use '--from-file' scenario. Also, first make sure that the consumer instances are inactive.
See <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-122%3A+Add+Reset+Consumer+Group+Offsets+tooling">KIP-122</a> for more details.
<p>
It has 3 execution options:
<ul>
<li>
(default) to display which offsets to reset.
</li>
<li>
--execute : to execute --reset-offsets process.
</li>
<li>
--export : to export the results to a CSV format.
</li>
</ul>
<p>
--reset-offsets also has following scenarios to choose from (at least one scenario must be selected):
<ul>
<li>
--to-datetime &lt;String: datetime&gt; : Reset offsets to offsets from datetime. Format: 'YYYY-MM-DDTHH:mm:SS.sss'
</li>
<li>
--to-earliest : Reset offsets to earliest offset.
</li>
<li>
--to-latest : Reset offsets to latest offset.
</li>
<li>
--shift-by &lt;Long: number-of-offsets&gt; : Reset offsets shifting current offset by 'n', where 'n' can be positive or negative.
</li>
<li>
--from-file : Reset offsets to values defined in CSV file.
</li>
<li>
--to-current : Resets offsets to current offset.
</li>
<li>
--by-duration &lt;String: duration&gt; : Reset offsets to offset by duration from current timestamp. Format: 'PnDTnHnMnS'
</li>
<li>
--to-offset : Reset offsets to a specific offset.
</li>
</ul>
Please note, that out of range offsets will be adjusted to available offset end. For example, if offset end is at 10 and offset shift request is
of 15, then, offset at 10 will actually be selected.
<p>
For example, to reset offsets of a consumer group to the latest offset:
<pre class="line-numbers"><code class="language-bash"> &gt; bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --reset-offsets --group consumergroup1 --topic topic1 --to-latest
TOPIC PARTITION NEW-OFFSET
topic1 0 0</code></pre>
<p>
If you are using the old high-level consumer and storing the group metadata in ZooKeeper (i.e. <code>offsets.storage=zookeeper</code>), pass
<code>--zookeeper</code> instead of <code>--bootstrap-server</code>:
<pre class="line-numbers"><code class="language-bash"> &gt; bin/kafka-consumer-groups.sh --zookeeper localhost:2181 --list</code></pre>
<h4 class="anchor-heading"><a id="basic_ops_cluster_expansion" class="anchor-link"></a><a href="#basic_ops_cluster_expansion">Expanding your cluster</a></h4>
Adding servers to a Kafka cluster is easy, just assign them a unique broker id and start up Kafka on your new servers. However these new servers will not automatically be assigned any data partitions, so unless partitions are moved to them they won't be doing any work until new topics are created. So usually when you add machines to your cluster you will want to migrate some existing data to these machines.
<p>
The process of migrating data is manually initiated but fully automated. Under the covers what happens is that Kafka will add the new server as a follower of the partition it is migrating and allow it to fully replicate the existing data in that partition. When the new server has fully replicated the contents of this partition and joined the in-sync replica one of the existing replicas will delete their partition's data.
<p>
The partition reassignment tool can be used to move partitions across brokers. An ideal partition distribution would ensure even data load and partition sizes across all brokers. The partition reassignment tool does not have the capability to automatically study the data distribution in a Kafka cluster and move partitions around to attain an even load distribution. As such, the admin has to figure out which topics or partitions should be moved around.
<p>
The partition reassignment tool can run in 3 mutually exclusive modes:
<ul>
<li>--generate: In this mode, given a list of topics and a list of brokers, the tool generates a candidate reassignment to move all partitions of the specified topics to the new brokers. This option merely provides a convenient way to generate a partition reassignment plan given a list of topics and target brokers.</li>
<li>--execute: In this mode, the tool kicks off the reassignment of partitions based on the user provided reassignment plan. (using the --reassignment-json-file option). This can either be a custom reassignment plan hand crafted by the admin or provided by using the --generate option</li>
<li>--verify: In this mode, the tool verifies the status of the reassignment for all partitions listed during the last --execute. The status can be either of successfully completed, failed or in progress</li>
</ul>
<h5 class="anchor-heading"><a id="basic_ops_automigrate" class="anchor-link"></a><a href="#basic_ops_automigrate">Automatically migrating data to new machines</a></h5>
The partition reassignment tool can be used to move some topics off of the current set of brokers to the newly added brokers. This is typically useful while expanding an existing cluster since it is easier to move entire topics to the new set of brokers, than moving one partition at a time. When used to do this, the user should provide a list of topics that should be moved to the new set of brokers and a target list of new brokers. The tool then evenly distributes all partitions for the given list of topics across the new set of brokers. During this move, the replication factor of the topic is kept constant. Effectively the replicas for all partitions for the input list of topics are moved from the old set of brokers to the newly added brokers.
<p>
For instance, the following example will move all partitions for topics foo1,foo2 to the new set of brokers 5,6. At the end of this move, all partitions for topics foo1 and foo2 will <i>only</i> exist on brokers 5,6.
<p>
Since the tool accepts the input list of topics as a json file, you first need to identify the topics you want to move and create the json file as follows:
<pre class="line-numbers"><code class="language-bash"> > cat topics-to-move.json
{"topics": [{"topic": "foo1"},
{"topic": "foo2"}],
"version":1
}</code></pre>
Once the json file is ready, use the partition reassignment tool to generate a candidate assignment:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --topics-to-move-json-file topics-to-move.json --broker-list "5,6" --generate
Current partition replica assignment
{"version":1,
"partitions":[{"topic":"foo1","partition":2,"replicas":[1,2]},
{"topic":"foo1","partition":0,"replicas":[3,4]},
{"topic":"foo2","partition":2,"replicas":[1,2]},
{"topic":"foo2","partition":0,"replicas":[3,4]},
{"topic":"foo1","partition":1,"replicas":[2,3]},
{"topic":"foo2","partition":1,"replicas":[2,3]}]
}
Proposed partition reassignment configuration
{"version":1,
"partitions":[{"topic":"foo1","partition":2,"replicas":[5,6]},
{"topic":"foo1","partition":0,"replicas":[5,6]},
{"topic":"foo2","partition":2,"replicas":[5,6]},
{"topic":"foo2","partition":0,"replicas":[5,6]},
{"topic":"foo1","partition":1,"replicas":[5,6]},
{"topic":"foo2","partition":1,"replicas":[5,6]}]
}</code></pre>
<p>
The tool generates a candidate assignment that will move all partitions from topics foo1,foo2 to brokers 5,6. Note, however, that at this point, the partition movement has not started, it merely tells you the current assignment and the proposed new assignment. The current assignment should be saved in case you want to rollback to it. The new assignment should be saved in a json file (e.g. expand-cluster-reassignment.json) to be input to the tool with the --execute option as follows:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file expand-cluster-reassignment.json --execute
Current partition replica assignment
{"version":1,
"partitions":[{"topic":"foo1","partition":2,"replicas":[1,2]},
{"topic":"foo1","partition":0,"replicas":[3,4]},
{"topic":"foo2","partition":2,"replicas":[1,2]},
{"topic":"foo2","partition":0,"replicas":[3,4]},
{"topic":"foo1","partition":1,"replicas":[2,3]},
{"topic":"foo2","partition":1,"replicas":[2,3]}]
}
Save this to use as the --reassignment-json-file option during rollback
Successfully started reassignment of partitions
{"version":1,
"partitions":[{"topic":"foo1","partition":2,"replicas":[5,6]},
{"topic":"foo1","partition":0,"replicas":[5,6]},
{"topic":"foo2","partition":2,"replicas":[5,6]},
{"topic":"foo2","partition":0,"replicas":[5,6]},
{"topic":"foo1","partition":1,"replicas":[5,6]},
{"topic":"foo2","partition":1,"replicas":[5,6]}]
}</code></pre>
<p>
Finally, the --verify option can be used with the tool to check the status of the partition reassignment. Note that the same expand-cluster-reassignment.json (used with the --execute option) should be used with the --verify option:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file expand-cluster-reassignment.json --verify
Status of partition reassignment:
Reassignment of partition [foo1,0] completed successfully
Reassignment of partition [foo1,1] is in progress
Reassignment of partition [foo1,2] is in progress
Reassignment of partition [foo2,0] completed successfully
Reassignment of partition [foo2,1] completed successfully
Reassignment of partition [foo2,2] completed successfully</code></pre>
<h5 class="anchor-heading"><a id="basic_ops_partitionassignment" class="anchor-link"></a><a href="#basic_ops_partitionassignment">Custom partition assignment and migration</a></h5>
The partition reassignment tool can also be used to selectively move replicas of a partition to a specific set of brokers. When used in this manner, it is assumed that the user knows the reassignment plan and does not require the tool to generate a candidate reassignment, effectively skipping the --generate step and moving straight to the --execute step
<p>
For instance, the following example moves partition 0 of topic foo1 to brokers 5,6 and partition 1 of topic foo2 to brokers 2,3:
<p>
The first step is to hand craft the custom reassignment plan in a json file:
<pre class="line-numbers"><code class="language-bash"> > cat custom-reassignment.json
{"version":1,"partitions":[{"topic":"foo1","partition":0,"replicas":[5,6]},{"topic":"foo2","partition":1,"replicas":[2,3]}]}</code></pre>
Then, use the json file with the --execute option to start the reassignment process:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file custom-reassignment.json --execute
Current partition replica assignment
{"version":1,
"partitions":[{"topic":"foo1","partition":0,"replicas":[1,2]},
{"topic":"foo2","partition":1,"replicas":[3,4]}]
}
Save this to use as the --reassignment-json-file option during rollback
Successfully started reassignment of partitions
{"version":1,
"partitions":[{"topic":"foo1","partition":0,"replicas":[5,6]},
{"topic":"foo2","partition":1,"replicas":[2,3]}]
}</code></pre>
<p>
The --verify option can be used with the tool to check the status of the partition reassignment. Note that the same custom-reassignment.json (used with the --execute option) should be used with the --verify option:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file custom-reassignment.json --verify
Status of partition reassignment:
Reassignment of partition [foo1,0] completed successfully
Reassignment of partition [foo2,1] completed successfully</code></pre>
<h4 class="anchor-heading"><a id="basic_ops_decommissioning_brokers" class="anchor-link"></a><a href="#basic_ops_decommissioning_brokers">Decommissioning brokers</a></h4>
The partition reassignment tool does not have the ability to automatically generate a reassignment plan for decommissioning brokers yet. As such, the admin has to come up with a reassignment plan to move the replica for all partitions hosted on the broker to be decommissioned, to the rest of the brokers. This can be relatively tedious as the reassignment needs to ensure that all the replicas are not moved from the decommissioned broker to only one other broker. To make this process effortless, we plan to add tooling support for decommissioning brokers in the future.
<h4 class="anchor-heading"><a id="basic_ops_increase_replication_factor" class="anchor-link"></a><a href="#basic_ops_increase_replication_factor">Increasing replication factor</a></h4>
Increasing the replication factor of an existing partition is easy. Just specify the extra replicas in the custom reassignment json file and use it with the --execute option to increase the replication factor of the specified partitions.
<p>
For instance, the following example increases the replication factor of partition 0 of topic foo from 1 to 3. Before increasing the replication factor, the partition's only replica existed on broker 5. As part of increasing the replication factor, we will add more replicas on brokers 6 and 7.
<p>
The first step is to hand craft the custom reassignment plan in a json file:
<pre class="line-numbers"><code class="language-bash"> > cat increase-replication-factor.json
{"version":1,
"partitions":[{"topic":"foo","partition":0,"replicas":[5,6,7]}]}</code></pre>
Then, use the json file with the --execute option to start the reassignment process:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file increase-replication-factor.json --execute
Current partition replica assignment
{"version":1,
"partitions":[{"topic":"foo","partition":0,"replicas":[5]}]}
Save this to use as the --reassignment-json-file option during rollback
Successfully started reassignment of partitions
{"version":1,
"partitions":[{"topic":"foo","partition":0,"replicas":[5,6,7]}]}</code></pre>
<p>
The --verify option can be used with the tool to check the status of the partition reassignment. Note that the same increase-replication-factor.json (used with the --execute option) should be used with the --verify option:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file increase-replication-factor.json --verify
Status of partition reassignment:
Reassignment of partition [foo,0] completed successfully</code></pre>
You can also verify the increase in replication factor with the kafka-topics tool:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-topics.sh --bootstrap-server localhost:9092 --topic foo --describe
Topic:foo PartitionCount:1 ReplicationFactor:3 Configs:
Topic: foo Partition: 0 Leader: 5 Replicas: 5,6,7 Isr: 5,6,7</code></pre>
<h4 class="anchor-heading"><a id="rep-throttle" class="anchor-link"></a><a href="#rep-throttle">Limiting Bandwidth Usage during Data Migration</a></h4>
Kafka lets you apply a throttle to replication traffic, setting an upper bound on the bandwidth used to move replicas from machine to machine. This is useful when rebalancing a cluster, bootstrapping a new broker or adding or removing brokers, as it limits the impact these data-intensive operations will have on users.
<p></p>
There are two interfaces that can be used to engage a throttle. The simplest, and safest, is to apply a throttle when invoking the kafka-reassign-partitions.sh, but kafka-configs.sh can also be used to view and alter the throttle values directly.
<p></p>
So for example, if you were to execute a rebalance, with the below command, it would move partitions at no more than 50MB/s.
<pre class="language-bash">$ bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --execute --reassignment-json-file bigger-cluster.json --throttle 50000000</code></pre>
When you execute this script you will see the throttle engage:
<pre class="line-numbers"><code class="language-bash"> The throttle limit was set to 50000000 B/s
Successfully started reassignment of partitions.</code></pre>
<p>Should you wish to alter the throttle, during a rebalance, say to increase the throughput so it completes quicker, you can do this by re-running the execute command passing the same reassignment-json-file:</p>
<pre class="language-bash">$ bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --execute --reassignment-json-file bigger-cluster.json --throttle 700000000
There is an existing assignment running.
The throttle limit was set to 700000000 B/s</code></pre>
<p>Once the rebalance completes the administrator can check the status of the rebalance using the --verify option.
If the rebalance has completed, the throttle will be removed via the --verify command. It is important that
administrators remove the throttle in a timely manner once rebalancing completes by running the command with
the --verify option. Failure to do so could cause regular replication traffic to be throttled. </p>
<p>When the --verify option is executed, and the reassignment has completed, the script will confirm that the throttle was removed:</p>
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --verify --reassignment-json-file bigger-cluster.json
Status of partition reassignment:
Reassignment of partition [my-topic,1] completed successfully
Reassignment of partition [mytopic,0] completed successfully
Throttle was removed.</code></pre>
<p>The administrator can also validate the assigned configs using the kafka-configs.sh. There are two pairs of throttle
configuration used to manage the throttling process. First pair refers to the throttle value itself. This is configured, at a broker
level, using the dynamic properties: </p>
<pre class="line-numbers"><code class="language-text"> leader.replication.throttled.rate
follower.replication.throttled.rate</code></pre>
<p>Then there is the configuration pair of enumerated sets of throttled replicas: </p>
<pre class="line-numbers"><code class="language-text"> leader.replication.throttled.replicas
follower.replication.throttled.replicas</code></pre>
<p>Which are configured per topic. </p>
<p>All four config values are automatically assigned by kafka-reassign-partitions.sh (discussed below).</p>
<p>To view the throttle limit configuration:</p>
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --describe --bootstrap-server localhost:9092 --entity-type brokers
Configs for brokers '2' are leader.replication.throttled.rate=700000000,follower.replication.throttled.rate=700000000
Configs for brokers '1' are leader.replication.throttled.rate=700000000,follower.replication.throttled.rate=700000000</code></pre>
<p>This shows the throttle applied to both leader and follower side of the replication protocol. By default both sides
are assigned the same throttled throughput value. </p>
<p>To view the list of throttled replicas:</p>
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --describe --bootstrap-server localhost:9092 --entity-type topics
Configs for topic 'my-topic' are leader.replication.throttled.replicas=1:102,0:101,
follower.replication.throttled.replicas=1:101,0:102</code></pre>
<p>Here we see the leader throttle is applied to partition 1 on broker 102 and partition 0 on broker 101. Likewise the
follower throttle is applied to partition 1 on
broker 101 and partition 0 on broker 102. </p>
<p>By default kafka-reassign-partitions.sh will apply the leader throttle to all replicas that exist before the
rebalance, any one of which might be leader.
It will apply the follower throttle to all move destinations. So if there is a partition with replicas on brokers
101,102, being reassigned to 102,103, a leader throttle,
for that partition, would be applied to 101,102 and a follower throttle would be applied to 103 only. </p>
<p>If required, you can also use the --alter switch on kafka-configs.sh to alter the throttle configurations manually.
</p>
<h5>Safe usage of throttled replication</h5>
<p>Some care should be taken when using throttled replication. In particular:</p>
<p><i>(1) Throttle Removal:</i></p>
The throttle should be removed in a timely manner once reassignment completes (by running kafka-reassign-partitions.sh
--verify).
<p><i>(2) Ensuring Progress:</i></p>
<p>If the throttle is set too low, in comparison to the incoming write rate, it is possible for replication to not
make progress. This occurs when:</p>
<pre>max(BytesInPerSec) > throttle</code></pre>
<p>
Where BytesInPerSec is the metric that monitors the write throughput of producers into each broker. </p>
<p>The administrator can monitor whether replication is making progress, during the rebalance, using the metric:</p>
<pre>kafka.server:type=FetcherLagMetrics,name=ConsumerLag,clientId=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)</code></pre>
<p>The lag should constantly decrease during replication. If the metric does not decrease the administrator should
increase the
throttle throughput as described above. </p>
<h4 class="anchor-heading"><a id="quotas" class="anchor-link"></a><a href="#quotas">Setting quotas</a></h4>
Quotas overrides and defaults may be configured at (user, client-id), user or client-id levels as described <a href="#design_quotas">here</a>.
By default, clients receive an unlimited quota.
It is possible to set custom quotas for each (user, client-id), user or client-id group.
<p>
Configure custom quota for (user=user1, client-id=clientA):
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type users --entity-name user1 --entity-type clients --entity-name clientA
Updated config for entity: user-principal 'user1', client-id 'clientA'.</code></pre>
Configure custom quota for user=user1:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type users --entity-name user1
Updated config for entity: user-principal 'user1'.</code></pre>
Configure custom quota for client-id=clientA:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type clients --entity-name clientA
Updated config for entity: client-id 'clientA'.</code></pre>
It is possible to set default quotas for each (user, client-id), user or client-id group by specifying <i>--entity-default</i> option instead of <i>--entity-name</i>.
<p>
Configure default client-id quota for user=userA:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type users --entity-name user1 --entity-type clients --entity-default
Updated config for entity: user-principal 'user1', default client-id.</code></pre>
Configure default quota for user:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type users --entity-default
Updated config for entity: default user-principal.</code></pre>
Configure default quota for client-id:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type clients --entity-default
Updated config for entity: default client-id.</code></pre>
Here's how to describe the quota for a given (user, client-id):
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --describe --entity-type users --entity-name user1 --entity-type clients --entity-name clientA
Configs for user-principal 'user1', client-id 'clientA' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200</code></pre>
Describe quota for a given user:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --describe --entity-type users --entity-name user1
Configs for user-principal 'user1' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200</code></pre>
Describe quota for a given client-id:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --describe --entity-type clients --entity-name clientA
Configs for client-id 'clientA' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200</code></pre>
If entity name is not specified, all entities of the specified type are described. For example, describe all users:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --describe --entity-type users
Configs for user-principal 'user1' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200
Configs for default user-principal are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200</code></pre>
Similarly for (user, client):
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --describe --entity-type users --entity-type clients
Configs for user-principal 'user1', default client-id are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200
Configs for user-principal 'user1', client-id 'clientA' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200</code></pre>
<p>
It is possible to set default quotas that apply to all client-ids by setting these configs on the brokers. These properties are applied only if quota overrides or defaults are not configured in Zookeeper. By default, each client-id receives an unlimited quota. The following sets the default quota per producer and consumer client-id to 10MB/sec.
<pre class="line-numbers"><code class="language-text"> quota.producer.default=10485760
quota.consumer.default=10485760</code></pre>
Note that these properties are being deprecated and may be removed in a future release. Defaults configured using kafka-configs.sh take precedence over these properties.
<h3 class="anchor-heading"><a id="datacenters" class="anchor-link"></a><a href="#datacenters">6.2 Datacenters</a></h3>
Some deployments will need to manage a data pipeline that spans multiple datacenters. Our recommended approach to this is to deploy a local Kafka cluster in each datacenter, with application instances in each datacenter interacting only with their local cluster and mirroring data between clusters (see the documentation on <a href="#georeplication">Geo-Replication</a> for how to do this).
<p>
This deployment pattern allows datacenters to act as independent entities and allows us to manage and tune inter-datacenter replication centrally. This allows each facility to stand alone and operate even if the inter-datacenter links are unavailable: when this occurs the mirroring falls behind until the link is restored at which time it catches up.
<p>
For applications that need a global view of all data you can use mirroring to provide clusters which have aggregate data mirrored from the local clusters in <i>all</i> datacenters. These aggregate clusters are used for reads by applications that require the full data set.
<p>
This is not the only possible deployment pattern. It is possible to read from or write to a remote Kafka cluster over the WAN, though obviously this will add whatever latency is required to get the cluster.
<p>
Kafka naturally batches data in both the producer and consumer so it can achieve high-throughput even over a high-latency connection. To allow this though it may be necessary to increase the TCP socket buffer sizes for the producer, consumer, and broker using the <code>socket.send.buffer.bytes</code> and <code>socket.receive.buffer.bytes</code> configurations. The appropriate way to set this is documented <a href="http://en.wikipedia.org/wiki/Bandwidth-delay_product">here</a>.
<p>
It is generally <i>not</i> advisable to run a <i>single</i> Kafka cluster that spans multiple datacenters over a high-latency link. This will incur very high replication latency both for Kafka writes and ZooKeeper writes, and neither Kafka nor ZooKeeper will remain available in all locations if the network between locations is unavailable.
<h3 class="anchor-heading"><a id="georeplication" class="anchor-link"></a><a href="#georeplication">6.3 Geo-Replication (Cross-Cluster Data Mirroring)</a></h3>
<h4 class="anchor-heading"><a id="georeplication-overview" class="anchor-link"></a><a href="#georeplication-overview">Geo-Replication Overview</a></h4>
<p>
Kafka administrators can define data flows that cross the boundaries of individual Kafka clusters, data centers, or geo-regions. Such event streaming setups are often needed for organizational, technical, or legal requirements. Common scenarios include:
</p>
<ul>
<li>Geo-replication</li>
<li>Disaster recovery</li>
<li>Feeding edge clusters into a central, aggregate cluster</li>
<li>Physical isolation of clusters (such as production vs. testing)</li>
<li>Cloud migration or hybrid cloud deployments</li>
<li>Legal and compliance requirements</li>
</ul>
<p>
Administrators can set up such inter-cluster data flows with Kafka's MirrorMaker (version 2), a tool to replicate data between different Kafka environments in a streaming manner. MirrorMaker is built on top of the Kafka Connect framework and supports features such as:
</p>
<ul>
<li>Replicates topics (data plus configurations)</li>
<li>Replicates consumer groups including offsets to migrate applications between clusters</li>
<li>Replicates ACLs</li>
<li>Preserves partitioning</li>
<li>Automatically detects new topics and partitions</li>
<li>Provides a wide range of metrics, such as end-to-end replication latency across multiple data centers/clusters</li>
<li>Fault-tolerant and horizontally scalable operations</li>
</ul>
<p>
<em>Note: Geo-replication with MirrorMaker replicates data across Kafka clusters. This inter-cluster replication is different from Kafka's <a href="#replication">intra-cluster replication</a>, which replicates data within the same Kafka cluster.</em>
</p>
<h4 class="anchor-heading"><a id="georeplication-flows" class="anchor-link"></a><a href="#georeplication-flows">What Are Replication Flows</a></h4>
<p>
With MirrorMaker, Kafka administrators can replicate topics, topic configurations, consumer groups and their offsets, and ACLs from one or more source Kafka clusters to one or more target Kafka clusters, i.e., across cluster environments. In a nutshell, MirrorMaker uses Connectors to consume from source clusters and produce to target clusters.
</p>
<p>
These directional flows from source to target clusters are called replication flows. They are defined with the format <code>{source_cluster}->{target_cluster}</code> in the MirrorMaker configuration file as described later. Administrators can create complex replication topologies based on these flows.
</p>
<p>
Here are some example patterns:
</p>
<ul>
<li>Active/Active high availability deployments: <code>A->B, B->A</code></li>
<li>Active/Passive or Active/Standby high availability deployments: <code>A->B</code></li>
<li>Aggregation (e.g., from many clusters to one): <code>A->K, B->K, C->K</code></li>
<li>Fan-out (e.g., from one to many clusters): <code>K->A, K->B, K->C</code></li>
<li>Forwarding: <code>A->B, B->C, C->D</code></li>
</ul>
<p>
By default, a flow replicates all topics and consumer groups. However, each replication flow can be configured independently. For instance, you can define that only specific topics or consumer groups are replicated from the source cluster to the target cluster.
</p>
<p>
Here is a first example on how to configure data replication from a <code>primary</code> cluster to a <code>secondary</code> cluster (an active/passive setup):
</p>
<pre class="line-numbers"><code class="language-text"># Basic settings
clusters = primary, secondary
primary.bootstrap.servers = broker3-primary:9092
secondary.bootstrap.servers = broker5-secondary:9092
# Define replication flows
primary->secondary.enable = true
primary->secondary.topics = foobar-topic, quux-.*
</code></pre>
<h4 class="anchor-heading"><a id="georeplication-mirrormaker" class="anchor-link"></a><a href="#georeplication-mirrormaker">Configuring Geo-Replication</a></h4>
<p>
The following sections describe how to configure and run a dedicated MirrorMaker cluster. If you want to run MirrorMaker within an existing Kafka Connect cluster or other supported deployment setups, please refer to <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-382%3A+MirrorMaker+2.0">KIP-382: MirrorMaker 2.0</a> and be aware that the names of configuration settings may vary between deployment modes.
</p>
<p>
Beyond what's covered in the following sections, further examples and information on configuration settings are available at:
</p>
<ul>
<li><a href="https://github.com/apache/kafka/blob/trunk/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java">MirrorMakerConfig</a>, <a href="https://github.com/apache/kafka/blob/trunk/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java">MirrorConnectorConfig</a></li>
<li><a href="https://github.com/apache/kafka/blob/trunk/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultTopicFilter.java">DefaultTopicFilter</a> for topics, <a href="https://github.com/apache/kafka/blob/trunk/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultGroupFilter.java">DefaultGroupFilter</a> for consumer groups</li>
<li>Example configuration settings in <a href="https://github.com/apache/kafka/blob/trunk/config/connect-mirror-maker.properties">connect-mirror-maker.properties</a>, <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-382%3A+MirrorMaker+2.0">KIP-382: MirrorMaker 2.0</a></li>
</ul>
<h5 class="anchor-heading"><a id="georeplication-config-syntax" class="anchor-link"></a><a href="#georeplication-config-syntax">Configuration File Syntax</a></h5>
<p>
The MirrorMaker configuration file is typically named <code>connect-mirror-maker.properties</code>. You can configure a variety of components in this file:
</p>
<ul>
<li>MirrorMaker settings: global settings including cluster definitions (aliases), plus custom settings per replication flow</li>
<li>Kafka Connect and connector settings</li>
<li>Kafka producer, consumer, and admin client settings</li>
</ul>
<p>
Example: Define MirrorMaker settings (explained in more detail later).
</p>
<pre class="line-numbers"><code class="language-text"># Global settings
clusters = us-west, us-east # defines cluster aliases
us-west.bootstrap.servers = broker3-west:9092
us-east.bootstrap.servers = broker5-east:9092
topics = .* # all topics to be replicated by default
# Specific replication flow settings (here: flow from us-west to us-east)
us-west->us-east.enable = true
us-west->us.east.topics = foo.*, bar.* # override the default above
</code></pre>
<p>
MirrorMaker is based on the Kafka Connect framework. Any Kafka Connect, source connector, and sink connector settings as described in the <a href="#connectconfigs">documentation chapter on Kafka Connect</a> can be used directly in the MirrorMaker configuration, without having to change or prefix the name of the configuration setting.
</p>
<p>
Example: Define custom Kafka Connect settings to be used by MirrorMaker.
</p>
<pre class="line-numbers"><code class="language-text"># Setting Kafka Connect defaults for MirrorMaker
tasks.max = 5
</code></pre>
<p>
Most of the default Kafka Connect settings work well for MirrorMaker out-of-the-box, with the exception of <code>tasks.max</code>. In order to evenly distribute the workload across more than one MirrorMaker process, it is recommended to set <code>tasks.max</code> to at least <code>2</code> (preferably higher) depending on the available hardware resources and the total number of topic-partitions to be replicated.
</p>
<p>
You can further customize MirrorMaker's Kafka Connect settings <em>per source or target cluster</em> (more precisely, you can specify Kafka Connect worker-level configuration settings "per connector"). Use the format of <code>{cluster}.{config_name}</code> in the MirrorMaker configuration file.
</p>
<p>
Example: Define custom connector settings for the <code>us-west</code> cluster.
</p>
<pre class="line-numbers"><code class="language-text"># us-west custom settings
us-west.offset.storage.topic = my-mirrormaker-offsets
</code></pre>
<p>
MirrorMaker internally uses the Kafka producer, consumer, and admin clients. Custom settings for these clients are often needed. To override the defaults, use the following format in the MirrorMaker configuration file:
</p>
<ul>
<li><code>{source}.consumer.{consumer_config_name}</code></li>
<li><code>{target}.producer.{producer_config_name}</code></li>
<li><code>{source_or_target}.admin.{admin_config_name}</code></li>
</ul>
<p>
Example: Define custom producer, consumer, admin client settings.
</p>
<pre class="line-numbers"><code class="language-text"># us-west cluster (from which to consume)
us-west.consumer.isolation.level = read_committed
us-west.admin.bootstrap.servers = broker57-primary:9092
# us-east cluster (to which to produce)
us-east.producer.compression.type = gzip
us-east.producer.buffer.memory = 32768
us-east.admin.bootstrap.servers = broker8-secondary:9092
</code></pre>
<h5 class="anchor-heading"><a id="georeplication-flow-create" class="anchor-link"></a><a href="#georeplication-flow-create">Creating and Enabling Replication Flows</a></h5>
<p>
To define a replication flow, you must first define the respective source and target Kafka clusters in the MirrorMaker configuration file.
</p>
<ul>
<li><code>clusters</code> (required): comma-separated list of Kafka cluster "aliases"</li>
<li><code>{clusterAlias}.bootstrap.servers</code> (required): connection information for the specific cluster; comma-separated list of "bootstrap" Kafka brokers
</ul>
<p>
Example: Define two cluster aliases <code>primary</code> and <code>secondary</code>, including their connection information.
</p>
<pre class="line-numbers"><code class="language-text">clusters = primary, secondary
primary.bootstrap.servers = broker10-primary:9092,broker-11-primary:9092
secondary.bootstrap.servers = broker5-secondary:9092,broker6-secondary:9092
</code></pre>
<p>
Secondly, you must explicitly enable individual replication flows with <code>{source}->{target}.enabled = true</code> as needed. Remember that flows are directional: if you need two-way (bidirectional) replication, you must enable flows in both directions.
</p>
<pre class="line-numbers"><code class="language-text"># Enable replication from primary to secondary
primary->secondary.enable = true
</code></pre>
<p>
By default, a replication flow will replicate all but a few special topics and consumer groups from the source cluster to the target cluster, and automatically detect any newly created topics and groups. The names of replicated topics in the target cluster will be prefixed with the name of the source cluster (see section further below). For example, the topic <code>foo</code> in the source cluster <code>us-west</code> would be replicated to a topic named <code>us-west.foo</code> in the target cluster <code>us-east</code>.
</p>
<p>
The subsequent sections explain how to customize this basic setup according to your needs.
</p>
<h5 class="anchor-heading"><a id="georeplication-flow-configure" class="anchor-link"></a><a href="#georeplication-flow-configure">Configuring Replication Flows</a></h5>
<p>
The configuration of a replication flow is a combination of top-level default settings (e.g., <code>topics</code>), on top of which flow-specific settings, if any, are applied (e.g., <code>us-west->us-east.topics</code>). To change the top-level defaults, add the respective top-level setting to the MirrorMaker configuration file. To override the defaults for a specific replication flow only, use the syntax format <code>{source}->{target}.{config.name}</code>.
</p>
<p>
The most important settings are:
</p>
<ul>
<li><code>topics</code>: list of topics or a regular expression that defines which topics in the source cluster to replicate (default: <code>topics = .*</code>)
<li><code>topics.exclude</code>: list of topics or a regular expression to subsequently exclude topics that were matched by the <code>topics</code> setting (default: <code>topics.exclude = .*[\-\.]internal, .*\.replica, __.*</code>)
<li><code>groups</code>: list of topics or regular expression that defines which consumer groups in the source cluster to replicate (default: <code>groups = .*</code>)
<li><code>groups.exclude</code>: list of topics or a regular expression to subsequently exclude consumer groups that were matched by the <code>groups</code> setting (default: <code>groups.exclude = console-consumer-.*, connect-.*, __.*</code>)
<li><code>{source}->{target}.enable</code>: set to <code>true</code> to enable the replication flow (default: <code>false</code>)
</ul>
<p>
Example:
</p>
<pre class="line-numbers"><code class="language-text"># Custom top-level defaults that apply to all replication flows
topics = .*
groups = consumer-group1, consumer-group2
# Don't forget to enable a flow!
us-west->us-east.enable = true
# Custom settings for specific replication flows
us-west->us-east.topics = foo.*
us-west->us-east.groups = bar.*
us-west->us-east.emit.heartbeats = false
</code></pre>
<p>
Additional configuration settings are supported, some of which are listed below. In most cases, you can leave these settings at their default values. See <a href="https://github.com/apache/kafka/blob/trunk/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java">MirrorMakerConfig</a> and <a href="https://github.com/apache/kafka/blob/trunk/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java">MirrorConnectorConfig</a> for further details.
</p>
<ul>
<li><code>refresh.topics.enabled</code>: whether to check for new topics in the source cluster periodically (default: true)
<li><code>refresh.topics.interval.seconds</code>: frequency of checking for new topics in the source cluster; lower values than the default may lead to performance degradation (default: 6000, every ten minutes)
<li><code>refresh.groups.enabled</code>: whether to check for new consumer groups in the source cluster periodically (default: true)
<li><code>refresh.groups.interval.seconds</code>: frequency of checking for new consumer groups in the source cluster; lower values than the default may lead to performance degradation (default: 6000, every ten minutes)
<li><code>sync.topic.configs.enabled</code>: whether to replicate topic configurations from the source cluster (default: true)
<li><code>sync.topic.acls.enabled</code>: whether to sync ACLs from the source cluster (default: true)
<li><code>emit.heartbeats.enabled</code>: whether to emit heartbeats periodically (default: true)
<li><code>emit.heartbeats.interval.seconds</code>: frequency at which heartbeats are emitted (default: 5, every five seconds)
<li><code>heartbeats.topic.replication.factor</code>: replication factor of MirrorMaker's internal heartbeat topics (default: 3)
<li><code>emit.checkpoints.enabled</code>: whether to emit MirrorMaker's consumer offsets periodically (default: true)
<li><code>emit.checkpoints.interval.seconds</code>: frequency at which checkpoints are emitted (default: 60, every minute)
<li><code>checkpoints.topic.replication.factor</code>: replication factor of MirrorMaker's internal checkpoints topics (default: 3)
<li><code>sync.group.offsets.enabled</code>: whether to periodically write the translated offsets of replicated consumer groups (in the source cluster) to <code>__consumer_offsets</code> topic in target cluster, as long as no active consumers in that group are connected to the target cluster (default: true)
<li><code>sync.group.offsets.interval.seconds</code>: frequency at which consumer group offsets are synced (default: 60, every minute)
<li><code>offset-syncs.topic.replication.factor</code>: replication factor of MirrorMaker's internal offset-sync topics (default: 3)
</ul>
<h5 class="anchor-heading"><a id="georeplication-flow-secure" class="anchor-link"></a><a href="#georeplication-flow-secure">Securing Replication Flows</a></h5>
<p>
MirrorMaker supports the same <a href="#connectconfigs">security settings as Kafka Connect</a>, so please refer to the linked section for further information.
</p>
<p>
Example: Encrypt communication between MirrorMaker and the <code>us-east</code> cluster.
</p>
<pre class="line-numbers"><code class="language-text">us-east.security.protocol=SSL
us-east.ssl.truststore.location=/path/to/truststore.jks
us-east.ssl.truststore.password=my-secret-password
us-east.ssl.keystore.location=/path/to/keystore.jks
us-east.ssl.keystore.password=my-secret-password
us-east.ssl.key.password=my-secret-password
</code></pre>
<h5 class="anchor-heading"><a id="georeplication-topic-naming" class="anchor-link"></a><a href="#georeplication-topic-naming">Custom Naming of Replicated Topics in Target Clusters</a></h5>
<p>
Replicated topics in a target cluster—sometimes called <em>remote</em> topics—are renamed according to a replication policy. MirrorMaker uses this policy to ensure that events (aka records, messages) from different clusters are not written to the same topic-partition. By default as per <a href="https://github.com/apache/kafka/blob/trunk/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/DefaultReplicationPolicy.java">DefaultReplicationPolicy</a>, the names of replicated topics in the target clusters have the format <code>{source}.{source_topic_name}</code>:
</p>
<pre class="line-numbers"><code class="language-text">us-west us-east
========= =================
bar-topic
foo-topic --> us-west.foo-topic
</code></pre>
<p>
You can customize the separator (default: <code>.</code>) with the <code>replication.policy.separator</code> setting:
</p>
<pre class="line-numbers"><code class="language-text"># Defining a custom separator
us-west->us-east.replication.policy.separator = _
</code></pre>
<p>
If you need further control over how replicated topics are named, you can implement a custom <code>ReplicationPolicy</code> and override <code>replication.policy.class</code> (default is <code>DefaultReplicationPolicy</code>) in the MirrorMaker configuration.
</p>
<h5 class="anchor-heading"><a id="georeplication-config-conflicts" class="anchor-link"></a><a href="#georeplication-config-conflicts">Preventing Configuration Conflicts</a></h5>
<p>
MirrorMaker processes share configuration via their target Kafka clusters. This behavior may cause conflicts when configurations differ among MirrorMaker processes that operate against the same target cluster.
</p>
<p>
For example, the following two MirrorMaker processes would be racy:
</p>
<pre class="line-numbers"><code class="language-text"># Configuration of process 1
A->B.enabled = true
A->B.topics = foo
# Configuration of process 2
A->B.enabled = true
A->B.topics = bar
</code></pre>
<p>
In this case, the two processes will share configuration via cluster <code>B</code>, which causes a conflict. Depending on which of the two processes is the elected "leader", the result will be that either the topic <code>foo</code> or the topic <code>bar</code> is replicated, but not both.
</p>
<p>
It is therefore important to keep the MirrorMaker configration consistent across replication flows to the same target cluster. This can be achieved, for example, through automation tooling or by using a single, shared MirrorMaker configuration file for your entire organization.
</p>
<h5 class="anchor-heading"><a id="georeplication-best-practice" class="anchor-link"></a><a href="#georeplication-best-practice">Best Practice: Consume from Remote, Produce to Local</a></h5>
<p>
To minimize latency ("producer lag"), it is recommended to locate MirrorMaker processes as close as possible to their target clusters, i.e., the clusters that it produces data to. That's because Kafka producers typically struggle more with unreliable or high-latency network connections than Kafka consumers.
</p>
<pre class="line-numbers"><code class="language-text">First DC Second DC
========== =========================
primary --------- MirrorMaker --> secondary
(remote) (local)
</code></pre>
<p>
To run such a "consume from remote, produce to local" setup, run the MirrorMaker processes close to and preferably in the same location as the target clusters, and explicitly set these "local" clusters in the <code>--clusters</code> command line parameter (blank-separated list of cluster aliases):
</p>
<pre class="line-numbers"><code class="language-text"># Run in secondary's data center, reading from the remote `primary` cluster
$ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties --clusters secondary
</code></pre>
The <code>--clusters secondary</code> tells the MirrorMaker process that the given cluster(s) are nearby, and prevents it from replicating data or sending configuration to clusters at other, remote locations.
<h5 class="anchor-heading"><a id="georeplication-example-active-passive" class="anchor-link"></a><a href="#georeplication-example-active-passive">Example: Active/Passive High Availability Deployment</a></h5>
<p>
The following example shows the basic settings to replicate topics from a primary to a secondary Kafka environment, but not from the secondary back to the primary. Please be aware that most production setups will need further configuration, such as security settings.
</p>
<pre class="line-numbers"><code class="language-text"># Unidirectional flow (one-way) from primary to secondary cluster
primary.bootstrap.servers = broker1-primary:9092
secondary.bootstrap.servers = broker2-secondary:9092
primary->secondary.enabled = true
secondary->primary.enabled = false
primary->secondary.topics = foo.* # only replicate some topics
</code></pre>
<h5 class="anchor-heading"><a id="georeplication-example-active-active" class="anchor-link"></a><a href="#georeplication-example-active-active">Example: Active/Active High Availability Deployment</a></h5>
<p>
The following example shows the basic settings to replicate topics between two clusters in both ways. Please be aware that most production setups will need further configuration, such as security settings.
</p>
<pre class="line-numbers"><code class="language-text"># Bidirectional flow (two-way) between us-west and us-east clusters
clusters = us-west, us-east
us-west.bootstrap.servers = broker1-west:9092,broker2-west:9092
Us-east.bootstrap.servers = broker3-east:9092,broker4-east:9092
us-west->us-east.enabled = true
us-east->us-west.enabled = true
</code></pre>
<p>
<em>Note on preventing replication "loops" (where topics will be originally replicated from A to B, then the replicated topics will be replicated yet again from B to A, and so forth)</em>: As long as you define the above flows in the same MirrorMaker configuration file, you do not need to explicitly add <code>topics.exclude</code> settings to prevent replication loops between the two clusters.
</p>
<h5 class="anchor-heading"><a id="georeplication-example-multi-cluster" class="anchor-link"></a><a href="#georeplication-example-multi-cluster">Example: Multi-Cluster Geo-Replication</a></h5>
<p>
Let's put all the information from the previous sections together in a larger example. Imagine there are three data centers (west, east, north), with two Kafka clusters in each data center (e.g., <code>west-1</code>, <code>west-2</code>). The example in this section shows how to configure MirrorMaker (1) for Active/Active replication within each data center, as well as (2) for Cross Data Center Replication (XDCR).
</p>
<p>
First, define the source and target clusters along with their replication flows in the configuration:
</p>
<pre class="line-numbers"><code class="language-text"># Basic settings
clusters: west-1, west-2, east-1, east-2, north-1, north-2
west-1.bootstrap.servers = ...
west-2.bootstrap.servers = ...
east-1.bootstrap.servers = ...
east-2.bootstrap.servers = ...
north-1.bootstrap.servers = ...
north-2.bootstrap.servers = ...
# Replication flows for Active/Active in West DC
west-1->west-2.enabled = true
west-2->west-1.enabled = true
# Replication flows for Active/Active in East DC
east-1->east-2.enabled = true
east-2->east-1.enabled = true
# Replication flows for Active/Active in North DC
north-1->north-2.enabled = true
north-2->north-1.enabled = true
# Replication flows for XDCR via west-1, east-1, north-1
west-1->east-1.enabled = true
west-1->north-1.enabled = true
east-1->west-1.enabled = true
east-1->north-1.enabled = true
north-1->west-1.enabled = true
north-1->east-1.enabled = true
</code></pre>
<p>
Then, in each data center, launch one or more MirrorMaker as follows:
</p>
<pre class="line-numbers"><code class="language-text"># In West DC:
$ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties --clusters west-1 west-2
# In East DC:
$ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties --clusters east-1 east-2
# In North DC:
$ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties --clusters north-1 north-2
</code></pre>
<p>
With this configuration, records produced to any cluster will be replicated within the data center, as well as across to other data centers. By providing the <code>--clusters</code> parameter, we ensure that each MirrorMaker process produces data to nearby clusters only.
</p>
<p>
<em>Note:</em> The <code>--clusters</code> parameter is, technically, not required here. MirrorMaker will work fine without it. However, throughput may suffer from "producer lag" between data centers, and you may incur unnecessary data transfer costs.
</p>
<h4 class="anchor-heading"><a id="georeplication-starting" class="anchor-link"></a><a href="#georeplication-starting">Starting Geo-Replication</a></h4>
<p>
You can run as few or as many MirrorMaker processes (think: nodes, servers) as needed. Because MirrorMaker is based on Kafka Connect, MirrorMaker processes that are configured to replicate the same Kafka clusters run in a distributed setup: They will find each other, share configuration (see section below), load balance their work, and so on. If, for example, you want to increase the throughput of replication flows, one option is to run additional MirrorMaker processes in parallel.
</p>
<p>
To start a MirrorMaker process, run the command:
</p>
<pre class="line-numbers"><code class="language-text">$ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties
</code></pre>
<p>
After startup, it may take a few minutes until a MirrorMaker process first begins to replicate data.
</p>
<p>
Optionally, as described previously, you can set the parameter <code>--clusters</code> to ensure that the MirrorMaker process produces data to nearby clusters only.
</p>
<pre class="line-numbers"><code class="language-text"># Note: The cluster alias us-west must be defined in the configuration file
$ ./bin/connect-mirror-maker.sh connect-mirror-maker.properties \
--clusters us-west
</code></pre>
<p>
<em>Note when testing replication of consumer groups:</em> By default, MirrorMaker does not replicate consumer groups created by the <code>kafka-console-consumer.sh</code> tool, which you might use to test your MirrorMaker setup on the command line. If you do want to replicate these consumer groups as well, set the <code>groups.exclude</code> configuration accordingly (default: <code>groups.exclude = console-consumer-.*, connect-.*, __.*</code>). Remember to update the configuration again once you completed your testing.
</p>
<h4 class="anchor-heading"><a id="georeplication-stopping" class="anchor-link"></a><a href="#georeplication-stopping">Stopping Geo-Replication</a></h4>
<p>
You can stop a running MirrorMaker process by sending a SIGTERM signal with the command:
</p>
<pre class="line-numbers"><code class="language-text">$ kill &lt;MirrorMaker pid&gt;
</code></pre>
<h4 class="anchor-heading"><a id="georeplication-apply-config-changes" class="anchor-link"></a><a href="#georeplication-apply-config-changes">Applying Configuration Changes</a></h4>
<p>
To make configuration changes take effect, the MirrorMaker process(es) must be restarted.
</p>
<h4 class="anchor-heading"><a id="georeplication-monitoring" class="anchor-link"></a><a href="#georeplication-monitoring">Monitoring Geo-Replication</a></h4>
<p>
It is recommended to monitor MirrorMaker processes to ensure all defined replication flows are up and running correctly. MirrorMaker is built on the Connect framework and inherits all of Connect's metrics, such <code>source-record-poll-rate</code>. In addition, MirrorMaker produces its own metrics under the <code>kafka.connect.mirror</code> metric group. Metrics are tagged with the following properties:
</p>
<ul>
<li><code>source</code>: alias of source cluster (e.g., <code>primary</code>)</li>
<li><code>target</code>: alias of target cluster (e.g., <code>secondary</code>)</li>
<li><code>topic</code>: replicated topic on target cluster</li>
<li><code>partition</code>: partition being replicated</li>
</ul>
<p>
Metrics are tracked for each replicated topic. The source cluster can be inferred from the topic name. For example, replicating <code>topic1</code> from <code>primary->secondary</code> will yield metrics like:
</p>
<ul>
<li><code>target=secondary</code>
<li><code>topic=primary.topic1</code>
<li><code>partition=1</code>
</ul>
<p>
The following metrics are emitted:
</p>
<pre class="line-numbers"><code class="language-text"># MBean: kafka.connect.mirror:type=MirrorSourceConnector,target=([-.w]+),topic=([-.w]+),partition=([0-9]+)
record-count # number of records replicated source -> target
record-age-ms # age of records when they are replicated
record-age-ms-min
record-age-ms-max
record-age-ms-avg
replication-latency-ms # time it takes records to propagate source->target
replication-latency-ms-min
replication-latency-ms-max
replication-latency-ms-avg
byte-rate # average number of bytes/sec in replicated records
# MBean: kafka.connect.mirror:type=MirrorCheckpointConnector,source=([-.w]+),target=([-.w]+)
checkpoint-latency-ms # time it takes to replicate consumer offsets
checkpoint-latency-ms-min
checkpoint-latency-ms-max
checkpoint-latency-ms-avg
</code></pre>
<p>
These metrics do not differentiate between created-at and log-append timestamps.
</p>
<h3 class="anchor-heading"><a id="config" class="anchor-link"></a><a href="#config">6.4 Kafka Configuration</a></h3>
<h4 class="anchor-heading"><a id="clientconfig" class="anchor-link"></a><a href="#clientconfig">Important Client Configurations</a></h4>
The most important producer configurations are:
<ul>
<li>acks</li>
<li>compression</li>
<li>batch size</li>
</ul>
The most important consumer configuration is the fetch size.
<p>
All configurations are documented in the <a href="#configuration">configuration</a> section.
<p>
<h4 class="anchor-heading"><a id="prodconfig" class="anchor-link"></a><a href="#prodconfig">A Production Server Config</a></h4>
Here is an example production server configuration:
<pre class="line-numbers"><code class="language-text"> # ZooKeeper
zookeeper.connect=[list of ZooKeeper servers]
# Log configuration
num.partitions=8
default.replication.factor=3
log.dir=[List of directories. Kafka should have its own dedicated disk(s) or SSD(s).]
# Other configurations
broker.id=[An integer. Start with 0 and increment by 1 for each new broker.]
listeners=[list of listeners]
auto.create.topics.enable=false
min.insync.replicas=2
queued.max.requests=[number of concurrent requests]</code></pre>
Our client configuration varies a fair amount between different use cases.
<h3 class="anchor-heading"><a id="java" class="anchor-link"></a><a href="#java">6.5 Java Version</a></h3>
Java 8 and Java 11 are supported. Java 11 performs significantly better if TLS is enabled, so it is highly recommended (it also includes a number of other
performance improvements: G1GC, CRC32C, Compact Strings, Thread-Local Handshakes and more).
From a security perspective, we recommend the latest released patch version as older freely available versions have disclosed security vulnerabilities.
Typical arguments for running Kafka with OpenJDK-based Java implementations (including Oracle JDK) are:
<pre class="line-numbers"><code class="language-text"> -Xmx6g -Xms6g -XX:MetaspaceSize=96m -XX:+UseG1GC
-XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:G1HeapRegionSize=16M
-XX:MinMetaspaceFreeRatio=50 -XX:MaxMetaspaceFreeRatio=80 -XX:+ExplicitGCInvokesConcurrent</code></pre>
For reference, here are the stats for one of LinkedIn's busiest clusters (at peak) that uses said Java arguments:
<ul>
<li>60 brokers</li>
<li>50k partitions (replication factor 2)</li>
<li>800k messages/sec in</li>
<li>300 MB/sec inbound, 1 GB/sec+ outbound</li>
</ul>
All of the brokers in that cluster have a 90% GC pause time of about 21ms with less than 1 young GC per second.
<h3 class="anchor-heading"><a id="hwandos" class="anchor-link"></a><a href="#hwandos">6.6 Hardware and OS</a></h3>
We are using dual quad-core Intel Xeon machines with 24GB of memory.
<p>
You need sufficient memory to buffer active readers and writers. You can do a back-of-the-envelope estimate of memory needs by assuming you want to be able to buffer for 30 seconds and compute your memory need as write_throughput*30.
<p>
The disk throughput is important. We have 8x7200 rpm SATA drives. In general disk throughput is the performance bottleneck, and more disks is better. Depending on how you configure flush behavior you may or may not benefit from more expensive disks (if you force flush often then higher RPM SAS drives may be better).
<h4 class="anchor-heading"><a id="os" class="anchor-link"></a><a href="#os">OS</a></h4>
Kafka should run well on any unix system and has been tested on Linux and Solaris.
<p>
We have seen a few issues running on Windows and Windows is not currently a well supported platform though we would be happy to change that.
<p>
It is unlikely to require much OS-level tuning, but there are three potentially important OS-level configurations:
<ul>
<li>File descriptor limits: Kafka uses file descriptors for log segments and open connections. If a broker hosts many partitions, consider that the broker needs at least (number_of_partitions)*(partition_size/segment_size) to track all log segments in addition to the number of connections the broker makes. We recommend at least 100000 allowed file descriptors for the broker processes as a starting point. Note: The mmap() function adds an extra reference to the file associated with the file descriptor fildes which is not removed by a subsequent close() on that file descriptor. This reference is removed when there are no more mappings to the file.
<li>Max socket buffer size: can be increased to enable high-performance data transfer between data centers as <a href="http://www.psc.edu/index.php/networking/641-tcp-tune">described here</a>.
<li>Maximum number of memory map areas a process may have (aka vm.max_map_count). <a href="http://kernel.org/doc/Documentation/sysctl/vm.txt">See the Linux kernel documentation</a>. You should keep an eye at this OS-level property when considering the maximum number of partitions a broker may have. By default, on a number of Linux systems, the value of vm.max_map_count is somewhere around 65535. Each log segment, allocated per partition, requires a pair of index/timeindex files, and each of these files consumes 1 map area. In other words, each log segment uses 2 map areas. Thus, each partition requires minimum 2 map areas, as long as it hosts a single log segment. That is to say, creating 50000 partitions on a broker will result allocation of 100000 map areas and likely cause broker crash with OutOfMemoryError (Map failed) on a system with default vm.max_map_count. Keep in mind that the number of log segments per partition varies depending on the segment size, load intensity, retention policy and, generally, tends to be more than one.
</ul>
<p>
<h4 class="anchor-heading"><a id="diskandfs" class="anchor-link"></a><a href="#diskandfs">Disks and Filesystem</a></h4>
We recommend using multiple drives to get good throughput and not sharing the same drives used for Kafka data with application logs or other OS filesystem activity to ensure good latency. You can either RAID these drives together into a single volume or format and mount each drive as its own directory. Since Kafka has replication the redundancy provided by RAID can also be provided at the application level. This choice has several tradeoffs.
<p>
If you configure multiple data directories partitions will be assigned round-robin to data directories. Each partition will be entirely in one of the data directories. If data is not well balanced among partitions this can lead to load imbalance between disks.
<p>
RAID can potentially do better at balancing load between disks (although it doesn't always seem to) because it balances load at a lower level. The primary downside of RAID is that it is usually a big performance hit for write throughput and reduces the available disk space.
<p>
Another potential benefit of RAID is the ability to tolerate disk failures. However our experience has been that rebuilding the RAID array is so I/O intensive that it effectively disables the server, so this does not provide much real availability improvement.
<h4 class="anchor-heading"><a id="appvsosflush" class="anchor-link"></a><a href="#appvsosflush">Application vs. OS Flush Management</a></h4>
Kafka always immediately writes all data to the filesystem and supports the ability to configure the flush policy that controls when data is forced out of the OS cache and onto disk using the flush. This flush policy can be controlled to force data to disk after a period of time or after a certain number of messages has been written. There are several choices in this configuration.
<p>
Kafka must eventually call fsync to know that data was flushed. When recovering from a crash for any log segment not known to be fsync'd Kafka will check the integrity of each message by checking its CRC and also rebuild the accompanying offset index file as part of the recovery process executed on startup.
<p>
Note that durability in Kafka does not require syncing data to disk, as a failed node will always recover from its replicas.
<p>
We recommend using the default flush settings which disable application fsync entirely. This means relying on the background flush done by the OS and Kafka's own background flush. This provides the best of all worlds for most uses: no knobs to tune, great throughput and latency, and full recovery guarantees. We generally feel that the guarantees provided by replication are stronger than sync to local disk, however the paranoid still may prefer having both and application level fsync policies are still supported.
<p>
The drawback of using application level flush settings is that it is less efficient in its disk usage pattern (it gives the OS less leeway to re-order writes) and it can introduce latency as fsync in most Linux filesystems blocks writes to the file whereas the background flushing does much more granular page-level locking.
<p>
In general you don't need to do any low-level tuning of the filesystem, but in the next few sections we will go over some of this in case it is useful.
<h4 class="anchor-heading"><a id="linuxflush" class="anchor-link"></a><a href="#linuxflush">Understanding Linux OS Flush Behavior</a></h4>
In Linux, data written to the filesystem is maintained in <a href="http://en.wikipedia.org/wiki/Page_cache">pagecache</a> until it must be written out to disk (due to an application-level fsync or the OS's own flush policy). The flushing of data is done by a set of background threads called pdflush (or in post 2.6.32 kernels "flusher threads").
<p>
Pdflush has a configurable policy that controls how much dirty data can be maintained in cache and for how long before it must be written back to disk.
This policy is described <a href="http://web.archive.org/web/20160518040713/http://www.westnet.com/~gsmith/content/linux-pdflush.htm">here</a>.
When Pdflush cannot keep up with the rate of data being written it will eventually cause the writing process to block incurring latency in the writes to slow down the accumulation of data.
<p>
You can see the current state of OS memory usage by doing
<pre class="language-bash"> &gt; cat /proc/meminfo </code></pre>
The meaning of these values are described in the link above.
<p>
Using pagecache has several advantages over an in-process cache for storing data that will be written out to disk:
<ul>
<li>The I/O scheduler will batch together consecutive small writes into bigger physical writes which improves throughput.
<li>The I/O scheduler will attempt to re-sequence writes to minimize movement of the disk head which improves throughput.
<li>It automatically uses all the free memory on the machine
</ul>
<h4 class="anchor-heading"><a id="filesystems" class="anchor-link"></a><a href="#filesystems">Filesystem Selection</a></h4>
<p>Kafka uses regular files on disk, and as such it has no hard dependency on a specific filesystem. The two filesystems which have the most usage, however, are EXT4 and XFS. Historically, EXT4 has had more usage, but recent improvements to the XFS filesystem have shown it to have better performance characteristics for Kafka's workload with no compromise in stability.</p>
<p>Comparison testing was performed on a cluster with significant message loads, using a variety of filesystem creation and mount options. The primary metric in Kafka that was monitored was the "Request Local Time", indicating the amount of time append operations were taking. XFS resulted in much better local times (160ms vs. 250ms+ for the best EXT4 configuration), as well as lower average wait times. The XFS performance also showed less variability in disk performance.</p>
<h5 class="anchor-heading"><a id="generalfs" class="anchor-link"></a><a href="#generalfs">General Filesystem Notes</a></h5>
For any filesystem used for data directories, on Linux systems, the following options are recommended to be used at mount time:
<ul>
<li>noatime: This option disables updating of a file's atime (last access time) attribute when the file is read. This can eliminate a significant number of filesystem writes, especially in the case of bootstrapping consumers. Kafka does not rely on the atime attributes at all, so it is safe to disable this.</li>
</ul>
<h5 class="anchor-heading"><a id="xfs" class="anchor-link"></a><a href="#xfs">XFS Notes</a></h5>
The XFS filesystem has a significant amount of auto-tuning in place, so it does not require any change in the default settings, either at filesystem creation time or at mount. The only tuning parameters worth considering are:
<ul>
<li>largeio: This affects the preferred I/O size reported by the stat call. While this can allow for higher performance on larger disk writes, in practice it had minimal or no effect on performance.</li>
<li>nobarrier: For underlying devices that have battery-backed cache, this option can provide a little more performance by disabling periodic write flushes. However, if the underlying device is well-behaved, it will report to the filesystem that it does not require flushes, and this option will have no effect.</li>
</ul>
<h5 class="anchor-heading"><a id="ext4" class="anchor-link"></a><a href="#ext4">EXT4 Notes</a></h5>
EXT4 is a serviceable choice of filesystem for the Kafka data directories, however getting the most performance out of it will require adjusting several mount options. In addition, these options are generally unsafe in a failure scenario, and will result in much more data loss and corruption. For a single broker failure, this is not much of a concern as the disk can be wiped and the replicas rebuilt from the cluster. In a multiple-failure scenario, such as a power outage, this can mean underlying filesystem (and therefore data) corruption that is not easily recoverable. The following options can be adjusted:
<ul>
<li>data=writeback: Ext4 defaults to data=ordered which puts a strong order on some writes. Kafka does not require this ordering as it does very paranoid data recovery on all unflushed log. This setting removes the ordering constraint and seems to significantly reduce latency.
<li>Disabling journaling: Journaling is a tradeoff: it makes reboots faster after server crashes but it introduces a great deal of additional locking which adds variance to write performance. Those who don't care about reboot time and want to reduce a major source of write latency spikes can turn off journaling entirely.
<li>commit=num_secs: This tunes the frequency with which ext4 commits to its metadata journal. Setting this to a lower value reduces the loss of unflushed data during a crash. Setting this to a higher value will improve throughput.
<li>nobh: This setting controls additional ordering guarantees when using data=writeback mode. This should be safe with Kafka as we do not depend on write ordering and improves throughput and latency.
<li>delalloc: Delayed allocation means that the filesystem avoid allocating any blocks until the physical write occurs. This allows ext4 to allocate a large extent instead of smaller pages and helps ensure the data is written sequentially. This feature is great for throughput. It does seem to involve some locking in the filesystem which adds a bit of latency variance.
</ul>
<h3 class="anchor-heading"><a id="monitoring" class="anchor-link"></a><a href="#monitoring">6.7 Monitoring</a></h3>
Kafka uses Yammer Metrics for metrics reporting in the server. The Java clients use Kafka Metrics, a built-in metrics registry that minimizes transitive dependencies pulled into client applications. Both expose metrics via JMX and can be configured to report stats using pluggable stats reporters to hook up to your monitoring system.
<p>
All Kafka rate metrics have a corresponding cumulative count metric with suffix <code>-total</code>. For example,
<code>records-consumed-rate</code> has a corresponding metric named <code>records-consumed-total</code>.
<p>
The easiest way to see the available metrics is to fire up jconsole and point it at a running kafka client or server; this will allow browsing all metrics with JMX.
<h4 class="anchor-heading"><a id="remote_jmx" class="anchor-link"></a><a href="#remote_jmx">Security Considerations for Remote Monitoring using JMX</a></h4>
Apache Kafka disables remote JMX by default. You can enable remote monitoring using JMX by setting the environment variable
<code>JMX_PORT</code> for processes started using the CLI or standard Java system properties to enable remote JMX programmatically.
You must enable security when enabling remote JMX in production scenarios to ensure that unauthorized users cannot monitor or
control your broker or application as well as the platform on which these are running. Note that authentication is disabled for
JMX by default in Kafka and security configs must be overridden for production deployments by setting the environment variable
<code>KAFKA_JMX_OPTS</code> for processes started using the CLI or by setting appropriate Java system properties. See
<a href="https://docs.oracle.com/javase/8/docs/technotes/guides/management/agent.html">Monitoring and Management Using JMX Technology</a>
for details on securing JMX.
<p>
We do graphing and alerting on the following metrics:
<table class="data-table">
<tbody><tr>
<th>Description</th>
<th>Mbean name</th>
<th>Normal value</th>
</tr>
<tr>
<td>Message in rate</td>
<td>kafka.server:type=BrokerTopicMetrics,name=MessagesInPerSec</td>
<td></td>
</tr>
<tr>
<td>Byte in rate from clients</td>
<td>kafka.server:type=BrokerTopicMetrics,name=BytesInPerSec</td>
<td></td>
</tr>
<tr>
<td>Byte in rate from other brokers</td>
<td>kafka.server:type=BrokerTopicMetrics,name=ReplicationBytesInPerSec</td>
<td></td>
</tr>
<tr>
<td>Request rate</td>
<td>kafka.network:type=RequestMetrics,name=RequestsPerSec,request={Produce|FetchConsumer|FetchFollower}</td>
<td></td>
</tr>
<tr>
<td>Error rate</td>
<td>kafka.network:type=RequestMetrics,name=ErrorsPerSec,request=([-.\w]+),error=([-.\w]+)</td>
<td>Number of errors in responses counted per-request-type, per-error-code. If a response contains
multiple errors, all are counted. error=NONE indicates successful responses.</td>
</tr>
<tr>
<td>Request size in bytes</td>
<td>kafka.network:type=RequestMetrics,name=RequestBytes,request=([-.\w]+)</td>
<td>Size of requests for each request type.</td>
</tr>
<tr>
<td>Temporary memory size in bytes</td>
<td>kafka.network:type=RequestMetrics,name=TemporaryMemoryBytes,request={Produce|Fetch}</td>
<td>Temporary memory used for message format conversions and decompression.</td>
</tr>
<tr>
<td>Message conversion time</td>
<td>kafka.network:type=RequestMetrics,name=MessageConversionsTimeMs,request={Produce|Fetch}</td>
<td>Time in milliseconds spent on message format conversions.</td>
</tr>
<tr>
<td>Message conversion rate</td>
<td>kafka.server:type=BrokerTopicMetrics,name={Produce|Fetch}MessageConversionsPerSec,topic=([-.\w]+)</td>
<td>Number of records which required message format conversion.</td>
</tr>
<tr>
<td>Request Queue Size</td>
<td>kafka.network:type=RequestChannel,name=RequestQueueSize</td>
<td>Size of the request queue.</td>
</tr>
<tr>
<td>Byte out rate to clients</td>
<td>kafka.server:type=BrokerTopicMetrics,name=BytesOutPerSec</td>
<td></td>
</tr>
<tr>
<td>Byte out rate to other brokers</td>
<td>kafka.server:type=BrokerTopicMetrics,name=ReplicationBytesOutPerSec</td>
<td></td>
</tr>
<tr>
<td>Message validation failure rate due to no key specified for compacted topic</td>
<td>kafka.server:type=BrokerTopicMetrics,name=NoKeyCompactedTopicRecordsPerSec</td>
<td></td>
</tr>
<tr>
<td>Message validation failure rate due to invalid magic number</td>
<td>kafka.server:type=BrokerTopicMetrics,name=InvalidMagicNumberRecordsPerSec</td>
<td></td>
</tr>
<tr>
<td>Message validation failure rate due to incorrect crc checksum</td>
<td>kafka.server:type=BrokerTopicMetrics,name=InvalidMessageCrcRecordsPerSec</td>
<td></td>
</tr>
<tr>
<td>Message validation failure rate due to non-continuous offset or sequence number in batch</td>
<td>kafka.server:type=BrokerTopicMetrics,name=InvalidOffsetOrSequenceRecordsPerSec</td>
<td></td>
</tr>
<tr>
<td>Log flush rate and time</td>
<td>kafka.log:type=LogFlushStats,name=LogFlushRateAndTimeMs</td>
<td></td>
</tr>
<tr>
<td># of under replicated partitions (the number of non-reassigning replicas - the number of ISR &gt 0)</td>
<td>kafka.server:type=ReplicaManager,name=UnderReplicatedPartitions</td>
<td>0</td>
</tr>
<tr>
<td># of under minIsr partitions (|ISR| &lt min.insync.replicas)</td>
<td>kafka.server:type=ReplicaManager,name=UnderMinIsrPartitionCount</td>
<td>0</td>
</tr>
<tr>
<td># of at minIsr partitions (|ISR| = min.insync.replicas)</td>
<td>kafka.server:type=ReplicaManager,name=AtMinIsrPartitionCount</td>
<td>0</td>
</tr>
<tr>
<td># of offline log directories</td>
<td>kafka.log:type=LogManager,name=OfflineLogDirectoryCount</td>
<td>0</td>
</tr>
<tr>
<td>Is controller active on broker</td>
<td>kafka.controller:type=KafkaController,name=ActiveControllerCount</td>
<td>only one broker in the cluster should have 1</td>
</tr>
<tr>
<td>Leader election rate</td>
<td>kafka.controller:type=ControllerStats,name=LeaderElectionRateAndTimeMs</td>
<td>non-zero when there are broker failures</td>
</tr>
<tr>
<td>Unclean leader election rate</td>
<td>kafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec</td>
<td>0</td>
</tr>
<tr>
<td>Pending topic deletes</td>
<td>kafka.controller:type=KafkaController,name=TopicsToDeleteCount</td>
<td></td>
</tr>
<tr>
<td>Pending replica deletes</td>
<td>kafka.controller:type=KafkaController,name=ReplicasToDeleteCount</td>
<td></td>
</tr>
<tr>
<td>Ineligible pending topic deletes</td>
<td>kafka.controller:type=KafkaController,name=TopicsIneligibleToDeleteCount</td>
<td></td>
</tr>
<tr>
<td>Ineligible pending replica deletes</td>
<td>kafka.controller:type=KafkaController,name=ReplicasIneligibleToDeleteCount</td>
<td></td>
</tr>
<tr>
<td>Partition counts</td>
<td>kafka.server:type=ReplicaManager,name=PartitionCount</td>
<td>mostly even across brokers</td>
</tr>
<tr>
<td>Leader replica counts</td>
<td>kafka.server:type=ReplicaManager,name=LeaderCount</td>
<td>mostly even across brokers</td>
</tr>
<tr>
<td>ISR shrink rate</td>
<td>kafka.server:type=ReplicaManager,name=IsrShrinksPerSec</td>
<td>If a broker goes down, ISR for some of the partitions will
shrink. When that broker is up again, ISR will be expanded
once the replicas are fully caught up. Other than that, the
expected value for both ISR shrink rate and expansion rate is 0. </td>
</tr>
<tr>
<td>ISR expansion rate</td>
<td>kafka.server:type=ReplicaManager,name=IsrExpandsPerSec</td>
<td>See above</td>
</tr>
<tr>
<td>Max lag in messages btw follower and leader replicas</td>
<td>kafka.server:type=ReplicaFetcherManager,name=MaxLag,clientId=Replica</td>
<td>lag should be proportional to the maximum batch size of a produce request.</td>
</tr>
<tr>
<td>Lag in messages per follower replica</td>
<td>kafka.server:type=FetcherLagMetrics,name=ConsumerLag,clientId=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)</td>
<td>lag should be proportional to the maximum batch size of a produce request.</td>
</tr>
<tr>
<td>Requests waiting in the producer purgatory</td>
<td>kafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation=Produce</td>
<td>non-zero if ack=-1 is used</td>
</tr>
<tr>
<td>Requests waiting in the fetch purgatory</td>
<td>kafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation=Fetch</td>
<td>size depends on fetch.wait.max.ms in the consumer</td>
</tr>
<tr>
<td>Request total time</td>
<td>kafka.network:type=RequestMetrics,name=TotalTimeMs,request={Produce|FetchConsumer|FetchFollower}</td>
<td>broken into queue, local, remote and response send time</td>
</tr>
<tr>
<td>Time the request waits in the request queue</td>
<td>kafka.network:type=RequestMetrics,name=RequestQueueTimeMs,request={Produce|FetchConsumer|FetchFollower}</td>
<td></td>
</tr>
<tr>
<td>Time the request is processed at the leader</td>
<td>kafka.network:type=RequestMetrics,name=LocalTimeMs,request={Produce|FetchConsumer|FetchFollower}</td>
<td></td>
</tr>
<tr>
<td>Time the request waits for the follower</td>
<td>kafka.network:type=RequestMetrics,name=RemoteTimeMs,request={Produce|FetchConsumer|FetchFollower}</td>
<td>non-zero for produce requests when ack=-1</td>
</tr>
<tr>
<td>Time the request waits in the response queue</td>
<td>kafka.network:type=RequestMetrics,name=ResponseQueueTimeMs,request={Produce|FetchConsumer|FetchFollower}</td>
<td></td>
</tr>
<tr>
<td>Time to send the response</td>
<td>kafka.network:type=RequestMetrics,name=ResponseSendTimeMs,request={Produce|FetchConsumer|FetchFollower}</td>
<td></td>
</tr>
<tr>
<td>Number of messages the consumer lags behind the producer by. Published by the consumer, not broker.</td>
<td>kafka.consumer:type=consumer-fetch-manager-metrics,client-id={client-id} Attribute: records-lag-max</td>
<td></td>
</tr>
<tr>
<td>The average fraction of time the network processors are idle</td>
<td>kafka.network:type=SocketServer,name=NetworkProcessorAvgIdlePercent</td>
<td>between 0 and 1, ideally &gt 0.3</td>
</tr>
<tr>
<td>The number of connections disconnected on a processor due to a client not re-authenticating and then using the connection beyond its expiration time for anything other than re-authentication</td>
<td>kafka.server:type=socket-server-metrics,listener=[SASL_PLAINTEXT|SASL_SSL],networkProcessor=&lt;#&gt;,name=expired-connections-killed-count</td>
<td>ideally 0 when re-authentication is enabled, implying there are no longer any older, pre-2.2.0 clients connecting to this (listener, processor) combination</td>
</tr>
<tr>
<td>The total number of connections disconnected, across all processors, due to a client not re-authenticating and then using the connection beyond its expiration time for anything other than re-authentication</td>
<td>kafka.network:type=SocketServer,name=ExpiredConnectionsKilledCount</td>
<td>ideally 0 when re-authentication is enabled, implying there are no longer any older, pre-2.2.0 clients connecting to this broker</td>
</tr>
<tr>
<td>The average fraction of time the request handler threads are idle</td>
<td>kafka.server:type=KafkaRequestHandlerPool,name=RequestHandlerAvgIdlePercent</td>
<td>between 0 and 1, ideally &gt 0.3</td>
</tr>
<tr>
<td>Bandwidth quota metrics per (user, client-id), user or client-id</td>
<td>kafka.server:type={Produce|Fetch},user=([-.\w]+),client-id=([-.\w]+)</td>
<td>Two attributes. throttle-time indicates the amount of time in ms the client was throttled. Ideally = 0.
byte-rate indicates the data produce/consume rate of the client in bytes/sec.
For (user, client-id) quotas, both user and client-id are specified. If per-client-id quota is applied to the client, user is not specified. If per-user quota is applied, client-id is not specified.</td>
</tr>
<tr>
<td>Request quota metrics per (user, client-id), user or client-id</td>
<td>kafka.server:type=Request,user=([-.\w]+),client-id=([-.\w]+)</td>
<td>Two attributes. throttle-time indicates the amount of time in ms the client was throttled. Ideally = 0.
request-time indicates the percentage of time spent in broker network and I/O threads to process requests from client group.
For (user, client-id) quotas, both user and client-id are specified. If per-client-id quota is applied to the client, user is not specified. If per-user quota is applied, client-id is not specified.</td>
</tr>
<tr>
<td>Requests exempt from throttling</td>
<td>kafka.server:type=Request</td>
<td>exempt-throttle-time indicates the percentage of time spent in broker network and I/O threads to process requests
that are exempt from throttling.</td>
</tr>
<tr>
<td>ZooKeeper client request latency</td>
<td>kafka.server:type=ZooKeeperClientMetrics,name=ZooKeeperRequestLatencyMs</td>
<td>Latency in millseconds for ZooKeeper requests from broker.</td>
</tr>
<tr>
<td>ZooKeeper connection status</td>
<td>kafka.server:type=SessionExpireListener,name=SessionState</td>
<td>Connection status of broker's ZooKeeper session which may be one of
Disconnected|SyncConnected|AuthFailed|ConnectedReadOnly|SaslAuthenticated|Expired.</td>
</tr>
<tr>
<td>Max time to load group metadata</td>
<td>kafka.server:type=group-coordinator-metrics,name=partition-load-time-max</td>
<td>maximum time, in milliseconds, it took to load offsets and group metadata from the consumer offset partitions loaded in the last 30 seconds (including time spent waiting for the loading task to be scheduled)</td>
</tr>
<tr>
<td>Avg time to load group metadata</td>
<td>kafka.server:type=group-coordinator-metrics,name=partition-load-time-avg</td>
<td>average time, in milliseconds, it took to load offsets and group metadata from the consumer offset partitions loaded in the last 30 seconds (including time spent waiting for the loading task to be scheduled)</td>
</tr>
<tr>
<td>Max time to load transaction metadata</td>
<td>kafka.server:type=transaction-coordinator-metrics,name=partition-load-time-max</td>
<td>maximum time, in milliseconds, it took to load transaction metadata from the consumer offset partitions loaded in the last 30 seconds (including time spent waiting for the loading task to be scheduled)</td>
</tr>
<tr>
<td>Avg time to load transaction metadata</td>
<td>kafka.server:type=transaction-coordinator-metrics,name=partition-load-time-avg</td>
<td>average time, in milliseconds, it took to load transaction metadata from the consumer offset partitions loaded in the last 30 seconds (including time spent waiting for the loading task to be scheduled)</td>
</tr>
<tr>
<td>Consumer Group Offset Count</td>
<td>kafka.server:type=GroupMetadataManager,name=NumOffsets</td>
<td>Total number of committed offsets for Consumer Groups</td>
</tr>
<tr>
<td>Consumer Group Count</td>
<td>kafka.server:type=GroupMetadataManager,name=NumGroups</td>
<td>Total number of Consumer Groups</td>
</tr>
<tr>
<td>Consumer Group Count, per State</td>
<td>kafka.server:type=GroupMetadataManager,name=NumGroups[PreparingRebalance,CompletingRebalance,Empty,Stable,Dead]</td>
<td>The number of Consumer Groups in each state: PreparingRebalance, CompletingRebalance, Empty, Stable, Dead</td>
</tr>
<tr>
<td>Number of reassigning partitions</td>
<td>kafka.server:type=ReplicaManager,name=ReassigningPartitions</td>
<td>The number of reassigning leader partitions on a broker.</td>
</tr>
<tr>
<td>Outgoing byte rate of reassignment traffic</td>
<td>kafka.server:type=BrokerTopicMetrics,name=ReassignmentBytesOutPerSec</td>
<td></td>
</tr>
<tr>
<td>Incoming byte rate of reassignment traffic</td>
<td>kafka.server:type=BrokerTopicMetrics,name=ReassignmentBytesInPerSec</td>
<td></td>
</tr>
</tbody></table>
<h4><a id="selector_monitoring" href="#selector_monitoring">Common monitoring metrics for producer/consumer/connect/streams</a></h4>
The following metrics are available on producer/consumer/connector/streams instances. For specific metrics, please see following sections.
<table class="data-table">
<tbody>
<tr>
<th>Metric/Attribute name</th>
<th>Description</th>
<th>Mbean name</th>
</tr>
<tr>
<td>connection-close-rate</td>
<td>Connections closed per second in the window.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>connection-close-total</td>
<td>Total connections closed in the window.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>connection-creation-rate</td>
<td>New connections established per second in the window.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>connection-creation-total</td>
<td>Total new connections established in the window.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>network-io-rate</td>
<td>The average number of network operations (reads or writes) on all connections per second.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>network-io-total</td>
<td>The total number of network operations (reads or writes) on all connections.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>outgoing-byte-rate</td>
<td>The average number of outgoing bytes sent per second to all servers.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>outgoing-byte-total</td>
<td>The total number of outgoing bytes sent to all servers.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>request-rate</td>
<td>The average number of requests sent per second.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>request-total</td>
<td>The total number of requests sent.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>request-size-avg</td>
<td>The average size of all requests in the window.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>request-size-max</td>
<td>The maximum size of any request sent in the window.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>incoming-byte-rate</td>
<td>Bytes/second read off all sockets.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>incoming-byte-total</td>
<td>Total bytes read off all sockets.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>response-rate</td>
<td>Responses received per second.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>response-total</td>
<td>Total responses received.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>select-rate</td>
<td>Number of times the I/O layer checked for new I/O to perform per second.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>select-total</td>
<td>Total number of times the I/O layer checked for new I/O to perform.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>io-wait-time-ns-avg</td>
<td>The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>io-wait-ratio</td>
<td>The fraction of time the I/O thread spent waiting.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>io-time-ns-avg</td>
<td>The average length of time for I/O per select call in nanoseconds.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>io-ratio</td>
<td>The fraction of time the I/O thread spent doing I/O.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>connection-count</td>
<td>The current number of active connections.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>successful-authentication-rate</td>
<td>Connections per second that were successfully authenticated using SASL or SSL.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>successful-authentication-total</td>
<td>Total connections that were successfully authenticated using SASL or SSL.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>failed-authentication-rate</td>
<td>Connections per second that failed authentication.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>failed-authentication-total</td>
<td>Total connections that failed authentication.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>successful-reauthentication-rate</td>
<td>Connections per second that were successfully re-authenticated using SASL.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>successful-reauthentication-total</td>
<td>Total connections that were successfully re-authenticated using SASL.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>reauthentication-latency-max</td>
<td>The maximum latency in ms observed due to re-authentication.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>reauthentication-latency-avg</td>
<td>The average latency in ms observed due to re-authentication.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>failed-reauthentication-rate</td>
<td>Connections per second that failed re-authentication.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>failed-reauthentication-total</td>
<td>Total connections that failed re-authentication.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>successful-authentication-no-reauth-total</td>
<td>Total connections that were successfully authenticated by older, pre-2.2.0 SASL clients that do not support re-authentication. May only be non-zero </td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
</tbody>
</table>
<h4><a id="common_node_monitoring" href="#common_node_monitoring">Common Per-broker metrics for producer/consumer/connect/streams</a></h4>
The following metrics are available on producer/consumer/connector/streams instances. For specific metrics, please see following sections.
<table class="data-table">
<tbody>
<tr>
<th>Metric/Attribute name</th>
<th>Description</th>
<th>Mbean name</th>
</tr>
<tr>
<td>outgoing-byte-rate</td>
<td>The average number of outgoing bytes sent per second for a node.</td>
<td>kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>outgoing-byte-total</td>
<td>The total number of outgoing bytes sent for a node.</td>
<td>kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>request-rate</td>
<td>The average number of requests sent per second for a node.</td>
<td>kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>request-total</td>
<td>The total number of requests sent for a node.</td>
<td>kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>request-size-avg</td>
<td>The average size of all requests in the window for a node.</td>
<td>kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>request-size-max</td>
<td>The maximum size of any request sent in the window for a node.</td>
<td>kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>incoming-byte-rate</td>
<td>The average number of bytes received per second for a node.</td>
<td>kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>incoming-byte-total</td>
<td>The total number of bytes received for a node.</td>
<td>kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>request-latency-avg</td>
<td>The average request latency in ms for a node.</td>
<td>kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>request-latency-max</td>
<td>The maximum request latency in ms for a node.</td>
<td>kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>response-rate</td>
<td>Responses received per second for a node.</td>
<td>kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>response-total</td>
<td>Total responses received for a node.</td>
<td>kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
</tbody>
</table>
<h4 class="anchor-heading"><a id="producer_monitoring" class="anchor-link"></a><a href="#producer_monitoring">Producer monitoring</a></h4>
The following metrics are available on producer instances.
<table class="data-table">
<tbody><tr>
<th>Metric/Attribute name</th>
<th>Description</th>
<th>Mbean name</th>
</tr>
<tr>
<td>waiting-threads</td>
<td>The number of user threads blocked waiting for buffer memory to enqueue their records.</td>
<td>kafka.producer:type=producer-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>buffer-total-bytes</td>
<td>The maximum amount of buffer memory the client can use (whether or not it is currently used).</td>
<td>kafka.producer:type=producer-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>buffer-available-bytes</td>
<td>The total amount of buffer memory that is not being used (either unallocated or in the free list).</td>
<td>kafka.producer:type=producer-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>bufferpool-wait-time</td>
<td>The fraction of time an appender waits for space allocation.</td>
<td>kafka.producer:type=producer-metrics,client-id=([-.\w]+)</td>
</tr>
</tbody></table>
<h5 class="anchor-heading"><a id="producer_sender_monitoring" class="anchor-link"></a><a href="#producer_sender_monitoring">Producer Sender Metrics</a></h5>
<!--#include virtual="generated/producer_metrics.html" -->
<h4 class="anchor-heading"><a id="consumer_monitoring" class="anchor-link"></a><a href="#consumer_monitoring">consumer monitoring</a></h4>
The following metrics are available on consumer instances.
<table class="data-table">
<tbody>
<tr>
<th>Metric/Attribute name</th>
<th>Description</th>
<th>Mbean name</th>
</tr>
<tr>
<td>time-between-poll-avg</td>
<td>The average delay between invocations of poll().</td>
<td>kafka.consumer:type=consumer-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>time-between-poll-max</td>
<td>The max delay between invocations of poll().</td>
<td>kafka.consumer:type=consumer-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>last-poll-seconds-ago</td>
<td>The number of seconds since the last poll() invocation.</td>
<td>kafka.consumer:type=consumer-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>poll-idle-ratio-avg</td>
<td>The average fraction of time the consumer's poll() is idle as opposed to waiting for the user code to process records.</td>
<td>kafka.consumer:type=consumer-metrics,client-id=([-.\w]+)</td>
</tr>
</tbody>
</table>
<h5 class="anchor-heading"><a id="consumer_group_monitoring" class="anchor-link"></a><a href="#consumer_group_monitoring">Consumer Group Metrics</a></h5>
<table class="data-table">
<tbody>
<tr>
<th>Metric/Attribute name</th>
<th>Description</th>
<th>Mbean name</th>
</tr>
<tr>
<td>commit-latency-avg</td>
<td>The average time taken for a commit request</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>commit-latency-max</td>
<td>The max time taken for a commit request</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>commit-rate</td>
<td>The number of commit calls per second</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>commit-total</td>
<td>The total number of commit calls</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>assigned-partitions</td>
<td>The number of partitions currently assigned to this consumer</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>heartbeat-response-time-max</td>
<td>The max time taken to receive a response to a heartbeat request</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>heartbeat-rate</td>
<td>The average number of heartbeats per second</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>heartbeat-total</td>
<td>The total number of heartbeats</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>join-time-avg</td>
<td>The average time taken for a group rejoin</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>join-time-max</td>
<td>The max time taken for a group rejoin</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>join-rate</td>
<td>The number of group joins per second</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>join-total</td>
<td>The total number of group joins</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>sync-time-avg</td>
<td>The average time taken for a group sync</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>sync-time-max</td>
<td>The max time taken for a group sync</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>sync-rate</td>
<td>The number of group syncs per second</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>sync-total</td>
<td>The total number of group syncs</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>rebalance-latency-avg</td>
<td>The average time taken for a group rebalance</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>rebalance-latency-max</td>
<td>The max time taken for a group rebalance</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>rebalance-latency-total</td>
<td>The total time taken for group rebalances so far</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>rebalance-total</td>
<td>The total number of group rebalances participated</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>rebalance-rate-per-hour</td>
<td>The number of group rebalance participated per hour</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>failed-rebalance-total</td>
<td>The total number of failed group rebalances</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>failed-rebalance-rate-per-hour</td>
<td>The number of failed group rebalance event per hour</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>last-rebalance-seconds-ago</td>
<td>The number of seconds since the last rebalance event</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>last-heartbeat-seconds-ago</td>
<td>The number of seconds since the last controller heartbeat</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>partitions-revoked-latency-avg</td>
<td>The average time taken by the on-partitions-revoked rebalance listener callback</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>partitions-revoked-latency-max</td>
<td>The max time taken by the on-partitions-revoked rebalance listener callback</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>partitions-assigned-latency-avg</td>
<td>The average time taken by the on-partitions-assigned rebalance listener callback</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>partitions-assigned-latency-max</td>
<td>The max time taken by the on-partitions-assigned rebalance listener callback</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>partitions-lost-latency-avg</td>
<td>The average time taken by the on-partitions-lost rebalance listener callback</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>partitions-lost-latency-max</td>
<td>The max time taken by the on-partitions-lost rebalance listener callback</td>
<td>kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)</td>
</tr>
</tbody>
</table>
<h5 class="anchor-heading"><a id="consumer_fetch_monitoring" class="anchor-link"></a><a href="#consumer_fetch_monitoring">Consumer Fetch Metrics</a></h5>
<!--#include virtual="generated/consumer_metrics.html" -->
<h4 class="anchor-heading"><a id="connect_monitoring" class="anchor-link"></a><a href="#connect_monitoring">Connect Monitoring</a></h4>
A Connect worker process contains all the producer and consumer metrics as well as metrics specific to Connect.
The worker process itself has a number of metrics, while each connector and task have additional metrics.
<!--#include virtual="generated/connect_metrics.html" -->
<h4 class="anchor-heading"><a id="kafka_streams_monitoring" class="anchor-link"></a><a href="#kafka_streams_monitoring">Streams Monitoring</a></h4>
A Kafka Streams instance contains all the producer and consumer metrics as well as additional metrics specific to Streams.
By default Kafka Streams has metrics with two recording levels: <code>debug</code> and <code>info</code>.
<p>
Note that the metrics have a 4-layer hierarchy. At the top level there are client-level metrics for each started
Kafka Streams client. Each client has stream threads, with their own metrics. Each stream thread has tasks, with their
own metrics. Each task has a number of processor nodes, with their own metrics. Each task also has a number of state stores
and record caches, all with their own metrics.
</p>
Use the following configuration option to specify which metrics
you want collected:
<pre><code>metrics.recording.level="info"</code></pre>
<h5 class="anchor-heading"><a id="kafka_streams_client_monitoring" class="anchor-link"></a><a href="#kafka_streams_client_monitoring">Client Metrics</a></h5>
All of the following metrics have a recording level of <code>info</code>:
<table class="data-table">
<tbody>
<tr>
<th>Metric/Attribute name</th>
<th>Description</th>
<th>Mbean name</th>
</tr>
<tr>
<td>version</td>
<td>The version of the Kafka Streams client.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>commit-id</td>
<td>The version control commit ID of the Kafka Streams client.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>application-id</td>
<td>The application ID of the Kafka Streams client.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>topology-description</td>
<td>The description of the topology executed in the Kafka Streams client.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>state</td>
<td>The state of the Kafka Streams client.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr>
</tbody>
</table>
<h5 class="anchor-heading"><a id="kafka_streams_thread_monitoring" class="anchor-link"></a><a href="#kafka_streams_thread_monitoring">Thread Metrics</a></h5>
All of the following metrics have a recording level of <code>info</code>:
<table class="data-table">
<tbody>
<tr>
<th>Metric/Attribute name</th>
<th>Description</th>
<th>Mbean name</th>
</tr>
<tr>
<td>commit-latency-avg</td>
<td>The average execution time in ms, for committing, across all running tasks of this thread.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>commit-latency-max</td>
<td>The maximum execution time in ms, for committing, across all running tasks of this thread.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>poll-latency-avg</td>
<td>The average execution time in ms, for consumer polling.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>poll-latency-max</td>
<td>The maximum execution time in ms, for consumer polling.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>process-latency-avg</td>
<td>The average execution time in ms, for processing.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>process-latency-max</td>
<td>The maximum execution time in ms, for processing.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>punctuate-latency-avg</td>
<td>The average execution time in ms, for punctuating.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>punctuate-latency-max</td>
<td>The maximum execution time in ms, for punctuating.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>commit-rate</td>
<td>The average number of commits per second.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>commit-total</td>
<td>The total number of commit calls.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>poll-rate</td>
<td>The average number of consumer poll calls per second.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>poll-total</td>
<td>The total number of consumer poll calls.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>process-rate</td>
<td>The average number of processed records per second.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>process-total</td>
<td>The total number of processed records.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>punctuate-rate</td>
<td>The average number of punctuate calls per second.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>punctuate-total</td>
<td>The total number of punctuate calls.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>task-created-rate</td>
<td>The average number of tasks created per second.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>task-created-total</td>
<td>The total number of tasks created.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>task-closed-rate</td>
<td>The average number of tasks closed per second.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
<tr>
<td>task-closed-total</td>
<td>The total number of tasks closed.</td>
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
</tr>
</tbody>
</table>
<h5 class="anchor-heading"><a id="kafka_streams_task_monitoring" class="anchor-link"></a><a href="#kafka_streams_task_monitoring">Task Metrics</a></h5>
All of the following metrics have a recording level of <code>debug</code>, except for metrics
dropped-records-rate and dropped-records-total which have a recording level of <code>info</code>:
<table class="data-table">
<tbody>
<tr>
<th>Metric/Attribute name</th>
<th>Description</th>
<th>Mbean name</th>
</tr>
<tr>
<td>process-latency-avg</td>
<td>The average execution time in ns, for processing.</td>
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
</tr>
<tr>
<td>process-latency-max</td>
<td>The maximum execution time in ns, for processing.</td>
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
</tr>
<tr>
<td>process-rate</td>
<td>The average number of processed records per second across all source processor nodes of this task.</td>
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
</tr>
<tr>
<td>process-total</td>
<td>The total number of processed records across all source processor nodes of this task.</td>
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
</tr>
<tr>
<td>commit-latency-avg</td>
<td>The average execution time in ns, for committing.</td>
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
</tr>
<tr>
<td>commit-latency-max</td>
<td>The maximum execution time in ns, for committing.</td>
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
</tr>
<tr>
<td>commit-rate</td>
<td>The average number of commit calls per second.</td>
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
</tr>
<tr>
<td>commit-total</td>
<td>The total number of commit calls. </td>
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
</tr>
<tr>
<td>record-lateness-avg</td>
<td>The average observed lateness of records (stream time - record timestamp).</td>
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
</tr>
<tr>
<td>record-lateness-max</td>
<td>The max observed lateness of records (stream time - record timestamp).</td>
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
</tr>
<tr>
<td>enforced-processing-rate</td>
<td>The average number of enforced processings per second.</td>
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
</tr>
<tr>
<td>enforced-processing-total</td>
<td>The total number enforced processings.</td>
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
</tr>
<tr>
<td>dropped-records-rate</td>
<td>The average number of records dropped within this task.</td>
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
</tr>
<tr>
<td>dropped-records-total</td>
<td>The total number of records dropped within this task.</td>
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
</tr>
</tbody>
</table>
<h5 class="anchor-heading"><a id="kafka_streams_node_monitoring" class="anchor-link"></a><a href="#kafka_streams_node_monitoring">Processor Node Metrics</a></h5>
The following metrics are only available on certain types of nodes, i.e., process-rate and process-total are
only available for source processor nodes and suppression-emit-rate and suppression-emit-total are only available
for suppression operation nodes. All of the metrics have a recording level of <code>debug</code>:
<table class="data-table">
<tbody>
<tr>
<th>Metric/Attribute name</th>
<th>Description</th>
<th>Mbean name</th>
</tr>
<tr>
<td>process-rate</td>
<td>The average number of records processed by a source processor node per second.</td>
<td>kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
</tr>
<tr>
<td>process-total</td>
<td>The total number of records processed by a source processor node per second.</td>
<td>kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
</tr>
<tr>
<td>suppression-emit-rate</td>
<td>The rate at which records that have been emitted downstream from suppression operation nodes.</td>
<td>kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
</tr>
<tr>
<td>suppression-emit-total</td>
<td>The total number of records that have been emitted downstream from suppression operation nodes.</td>
<td>kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
</tr>
</tbody>
</table>
<h5 class="anchor-heading"><a id="kafka_streams_store_monitoring" class="anchor-link"></a><a href="#kafka_streams_store_monitoring">State Store Metrics</a></h5>
All of the following metrics have a recording level of <code>debug</code>. Note that the <code>store-scope</code> value is specified in <code>StoreSupplier#metricsScope()</code> for user's customized
state stores; for built-in state stores, currently we have:
<ul>
<li><code>in-memory-state</code></li>
<li><code>in-memory-lru-state</code></li>
<li><code>in-memory-window-state</code></li>
<li><code>in-memory-suppression</code> (for suppression buffers)</li>
<li><code>rocksdb-state</code> (for RocksDB backed key-value store)</li>
<li><code>rocksdb-window-state</code> (for RocksDB backed window store)</li>
<li><code>rocksdb-session-state</code> (for RocksDB backed session store)</li>
</ul>
Metrics suppression-buffer-size-avg, suppression-buffer-size-max, suppression-buffer-count-avg, and suppression-buffer-count-max
are only available for suppression buffers. All other metrics are not available for suppression buffers.
<table class="data-table">
<tbody>
<tr>
<th>Metric/Attribute name</th>
<th>Description</th>
<th>Mbean name</th>
</tr>
<tr>
<td>put-latency-avg</td>
<td>The average put execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>put-latency-max</td>
<td>The maximum put execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>put-if-absent-latency-avg</td>
<td>The average put-if-absent execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>put-if-absent-latency-max</td>
<td>The maximum put-if-absent execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>get-latency-avg</td>
<td>The average get execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>get-latency-max</td>
<td>The maximum get execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>delete-latency-avg</td>
<td>The average delete execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>delete-latency-max</td>
<td>The maximum delete execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>put-all-latency-avg</td>
<td>The average put-all execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>put-all-latency-max</td>
<td>The maximum put-all execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>all-latency-avg</td>
<td>The average all operation execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>all-latency-max</td>
<td>The maximum all operation execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>range-latency-avg</td>
<td>The average range execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>range-latency-max</td>
<td>The maximum range execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>flush-latency-avg</td>
<td>The average flush execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>flush-latency-max</td>
<td>The maximum flush execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>restore-latency-avg</td>
<td>The average restore execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>restore-latency-max</td>
<td>The maximum restore execution time in ns. </td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>put-rate</td>
<td>The average put rate for this store.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>put-if-absent-rate</td>
<td>The average put-if-absent rate for this store.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>get-rate</td>
<td>The average get rate for this store.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>delete-rate</td>
<td>The average delete rate for this store.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>put-all-rate</td>
<td>The average put-all rate for this store.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>all-rate</td>
<td>The average all operation rate for this store.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>range-rate</td>
<td>The average range rate for this store.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>flush-rate</td>
<td>The average flush rate for this store.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>restore-rate</td>
<td>The average restore rate for this store.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>suppression-buffer-size-avg</td>
<td>The average total size, in bytes, of the buffered data over the sampling window.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),in-memory-suppression-id=([-.\w]+)</td>
</tr>
<tr>
<td>suppression-buffer-size-max</td>
<td>The maximum total size, in bytes, of the buffered data over the sampling window.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),in-memory-suppression-id=([-.\w]+)</td>
</tr>
<tr>
<td>suppression-buffer-count-avg</td>
<td>The average number of records buffered over the sampling window.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),in-memory-suppression-id=([-.\w]+)</td>
</tr>
<tr>
<td>suppression-buffer-count-max</td>
<td>The maximum number of records buffered over the sampling window.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),in-memory-suppression-id=([-.\w]+)</td>
</tr>
</tbody>
</table>
<h5 class="anchor-heading"><a id="kafka_streams_rocksdb_monitoring" class="anchor-link"></a><a href="#kafka_streams_rocksdb_monitoring">RocksDB Metrics</a></h5>
All of the following metrics have a recording level of <code>debug</code>.
The metrics are collected every minute from the RocksDB state stores.
If a state store consists of multiple RocksDB instances as it is the case for aggregations over time and session windows,
each metric reports an aggregation over the RocksDB instances of the state store.
Note that the <code>store-scope</code> for built-in RocksDB state stores are currently the following:
<ul>
<li><code>rocksdb-state</code> (for RocksDB backed key-value store)</li>
<li><code>rocksdb-window-state</code> (for RocksDB backed window store)</li>
<li><code>rocksdb-session-state</code> (for RocksDB backed session store)</li>
</ul>
<table class="data-table">
<tbody>
<tr>
<th>Metric/Attribute name</th>
<th>Description</th>
<th>Mbean name</th>
</tr>
<tr>
<td>bytes-written-rate</td>
<td>The average number of bytes written per second to the RocksDB state store.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>bytes-written-total</td>
<td>The total number of bytes written to the RocksDB state store.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>bytes-read-rate</td>
<td>The average number of bytes read per second from the RocksDB state store.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>bytes-read-total</td>
<td>The total number of bytes read from the RocksDB state store.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>memtable-bytes-flushed-rate</td>
<td>The average number of bytes flushed per second from the memtable to disk.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>memtable-bytes-flushed-total</td>
<td>The total number of bytes flushed from the memtable to disk.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>memtable-hit-ratio</td>
<td>The ratio of memtable hits relative to all lookups to the memtable.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>block-cache-data-hit-ratio</td>
<td>The ratio of block cache hits for data blocks relative to all lookups for data blocks to the block cache.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>block-cache-index-hit-ratio</td>
<td>The ratio of block cache hits for index blocks relative to all lookups for index blocks to the block cache.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>block-cache-filter-hit-ratio</td>
<td>The ratio of block cache hits for filter blocks relative to all lookups for filter blocks to the block cache.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>write-stall-duration-avg</td>
<td>The average duration of write stalls in ms.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>write-stall-duration-total</td>
<td>The total duration of write stalls in ms.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>bytes-read-compaction-rate</td>
<td>The average number of bytes read per second during compaction.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>bytes-written-compaction-rate</td>
<td>The average number of bytes written per second during compaction.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>number-open-files</td>
<td>The number of current open files.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>number-file-errors-total</td>
<td>The total number of file errors occurred.</td>
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
</tbody>
</table>
<h5 class="anchor-heading"><a id="kafka_streams_cache_monitoring" class="anchor-link"></a><a href="#kafka_streams_cache_monitoring">Record Cache Metrics</a></h5>
All of the following metrics have a recording level of <code>debug</code>:
<table class="data-table">
<tbody>
<tr>
<th>Metric/Attribute name</th>
<th>Description</th>
<th>Mbean name</th>
</tr>
<tr>
<td>hit-ratio-avg</td>
<td>The average cache hit ratio defined as the ratio of cache read hits over the total cache read requests.</td>
<td>kafka.streams:type=stream-record-cache-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),record-cache-id=([-.\w]+)</td>
</tr>
<tr>
<td>hit-ratio-min</td>
<td>The mininum cache hit ratio.</td>
<td>kafka.streams:type=stream-record-cache-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),record-cache-id=([-.\w]+)</td>
</tr>
<tr>
<td>hit-ratio-max</td>
<td>The maximum cache hit ratio.</td>
<td>kafka.streams:type=stream-record-cache-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),record-cache-id=([-.\w]+)</td>
</tr>
</tbody>
</table>
<h4 class="anchor-heading"><a id="others_monitoring" class="anchor-link"></a><a href="#others_monitoring">Others</a></h4>
We recommend monitoring GC time and other stats and various server stats such as CPU utilization, I/O service time, etc.
On the client side, we recommend monitoring the message/byte rate (global and per topic), request rate/size/time, and on the consumer side, max lag in messages among all partitions and min fetch request rate. For a consumer to keep up, max lag needs to be less than a threshold and min fetch rate needs to be larger than 0.
<h3 class="anchor-heading"><a id="zk" class="anchor-link"></a><a href="#zk">6.8 ZooKeeper</a></h3>
<h4 class="anchor-heading"><a id="zkversion" class="anchor-link"></a><a href="#zkversion">Stable version</a></h4>
The current stable branch is 3.5. Kafka is regularly updated to include the latest release in the 3.5 series.
<h4 class="anchor-heading"><a id="zkops" class="anchor-link"></a><a href="#zkops">Operationalizing ZooKeeper</a></h4>
Operationally, we do the following for a healthy ZooKeeper installation:
<ul>
<li>Redundancy in the physical/hardware/network layout: try not to put them all in the same rack, decent (but don't go nuts) hardware, try to keep redundant power and network paths, etc. A typical ZooKeeper ensemble has 5 or 7 servers, which tolerates 2 and 3 servers down, respectively. If you have a small deployment, then using 3 servers is acceptable, but keep in mind that you'll only be able to tolerate 1 server down in this case. </li>
<li>I/O segregation: if you do a lot of write type traffic you'll almost definitely want the transaction logs on a dedicated disk group. Writes to the transaction log are synchronous (but batched for performance), and consequently, concurrent writes can significantly affect performance. ZooKeeper snapshots can be one such a source of concurrent writes, and ideally should be written on a disk group separate from the transaction log. Snapshots are written to disk asynchronously, so it is typically ok to share with the operating system and message log files. You can configure a server to use a separate disk group with the dataLogDir parameter.</li>
<li>Application segregation: Unless you really understand the application patterns of other apps that you want to install on the same box, it can be a good idea to run ZooKeeper in isolation (though this can be a balancing act with the capabilities of the hardware).</li>
<li>Use care with virtualization: It can work, depending on your cluster layout and read/write patterns and SLAs, but the tiny overheads introduced by the virtualization layer can add up and throw off ZooKeeper, as it can be very time sensitive</li>
<li>ZooKeeper configuration: It's java, make sure you give it 'enough' heap space (We usually run them with 3-5G, but that's mostly due to the data set size we have here). Unfortunately we don't have a good formula for it, but keep in mind that allowing for more ZooKeeper state means that snapshots can become large, and large snapshots affect recovery time. In fact, if the snapshot becomes too large (a few gigabytes), then you may need to increase the initLimit parameter to give enough time for servers to recover and join the ensemble.</li>
<li>Monitoring: Both JMX and the 4 letter words (4lw) commands are very useful, they do overlap in some cases (and in those cases we prefer the 4 letter commands, they seem more predictable, or at the very least, they work better with the LI monitoring infrastructure)</li>
<li>Don't overbuild the cluster: large clusters, especially in a write heavy usage pattern, means a lot of intracluster communication (quorums on the writes and subsequent cluster member updates), but don't underbuild it (and risk swamping the cluster). Having more servers adds to your read capacity.</li>
</ul>
Overall, we try to keep the ZooKeeper system as small as will handle the load (plus standard growth capacity planning) and as simple as possible. We try not to do anything fancy with the configuration or application layout as compared to the official release as well as keep it as self contained as possible. For these reasons, we tend to skip the OS packaged versions, since it has a tendency to try to put things in the OS standard hierarchy, which can be 'messy', for want of a better way to word it.
</script>
<div class="p-ops"></div>