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.
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.
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 <ahref="#topicconfigs">topic configurations</a> used for auto-created topics.
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.
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 <ahref="#intro_consumers">concepts section</a>.
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.
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 <ahref="#topicconfigs">here</a>.
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.
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:
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.
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.
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 preferred replicas. This behaviour is configured with:
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:
When a topic is <ahref="#basic_ops_add_topic">created</a>, <ahref="#basic_ops_modify_topic">modified</a> or replicas are <ahref="#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.
<h4class="anchor-heading"><aid="basic_ops_mirror_maker"class="anchor-link"></a><ahref="#basic_ops_mirror_maker">Mirroring data between clusters & Geo-replication</a></h4>
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 <ahref="#georeplication">Geo-Replication</a> for further information.
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:
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.
<li>--members --verbose: On top of the information reported by the "--members" options above, this option also provides the partitions assigned to each member.
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 <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-122%3A+Add+Reset+Consumer+Group+Offsets+tooling">KIP-122</a> for more details.
<h4class="anchor-heading"><aid="basic_ops_cluster_expansion"class="anchor-link"></a><ahref="#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>
<h5class="anchor-heading"><aid="basic_ops_automigrate"class="anchor-link"></a><ahref="#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:
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:
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:
<h5class="anchor-heading"><aid="basic_ops_partitionassignment"class="anchor-link"></a><ahref="#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:
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:
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.
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:
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:
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.
<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>
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>.
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.
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.
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 <ahref="#georeplication">Geo-Replication</a> for how to do this).
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 <ahref="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.
<h3class="anchor-heading"><aid="georeplication"class="anchor-link"></a><ahref="#georeplication">6.3 Geo-Replication (Cross-Cluster Data Mirroring)</a></h3>
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 <ahref="#replication">intra-cluster replication</a>, which replicates data within the same Kafka cluster.</em>
</p>
<h4class="anchor-heading"><aid="georeplication-flows"class="anchor-link"></a><ahref="#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>
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):
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 <ahref="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:
<li><ahref="https://github.com/apache/kafka/blob/trunk/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultTopicFilter.java">DefaultTopicFilter</a> for topics, <ahref="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 <ahref="https://github.com/apache/kafka/blob/trunk/config/connect-mirror-maker.properties">connect-mirror-maker.properties</a>, <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-382%3A+MirrorMaker+2.0">KIP-382: MirrorMaker 2.0</a></li>
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>
<preclass="line-numbers"><codeclass="language-text"># Global settings
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 <ahref="#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>
<preclass="line-numbers"><codeclass="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.
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:
<h5class="anchor-heading"><aid="georeplication-flow-create"class="anchor-link"></a><ahref="#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.
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>
<preclass="line-numbers"><codeclass="language-text"># Enable replication from primary to secondary
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.
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>
<preclass="line-numbers"><codeclass="language-text"># Custom top-level defaults that apply to all replication flows
Additional configuration settings are supported, some of which are listed below. In most cases, you can leave these settings at their default values. See <ahref="https://github.com/apache/kafka/blob/trunk/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java">MirrorMakerConfig</a> and <ahref="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: 600, every ten minutes)
<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: 600, every ten minutes)
<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: false)
MirrorMaker supports the same <ahref="#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.
<h5class="anchor-heading"><aid="georeplication-topic-naming"class="anchor-link"></a><ahref="#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 <ahref="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>:
You can customize the separator (default: <code>.</code>) with the <code>replication.policy.separator</code> setting:
</p>
<preclass="line-numbers"><codeclass="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.
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>
<preclass="line-numbers"><codeclass="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>
<h5class="anchor-heading"><aid="georeplication-best-practice"class="anchor-link"></a><ahref="#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>
<preclass="line-numbers"><codeclass="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>
<preclass="line-numbers"><codeclass="language-text"># Run in secondary's data center, reading from the remote `primary` cluster
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.
<h5class="anchor-heading"><aid="georeplication-example-active-passive"class="anchor-link"></a><ahref="#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>
<preclass="line-numbers"><codeclass="language-text"># Unidirectional flow (one-way) from primary to secondary cluster
primary->secondary.topics = foo.* # only replicate some topics
</code></pre>
<h5class="anchor-heading"><aid="georeplication-example-active-active"class="anchor-link"></a><ahref="#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>
<preclass="line-numbers"><codeclass="language-text"># Bidirectional flow (two-way) between us-west and us-east clusters
<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.
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:
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.
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.
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>
<preclass="line-numbers"><codeclass="language-text"># Note: The cluster alias us-west must be defined in the configuration file
<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.
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:
As a highly scalable event streaming platform, Kafka is used by many users as their central nervous system, connecting in real-time a wide range of different systems and applications from various teams and lines of businesses. Such multi-tenant cluster environments command proper control and management to ensure the peaceful coexistence of these different needs. This section highlights features and best practices to set up such shared environments, which should help you operate clusters that meet SLAs/OLAs and that minimize potential collateral damage caused by "noisy neighbors".
</p>
<p>
Multi-tenancy is a many-sided subject, including but not limited to:
</p>
<ul>
<li>Creating user spaces for tenants (sometimes called namespaces)</li>
<li>Configuring topics with data retention policies and more</li>
<li>Securing topics and clusters with encryption, authentication, and authorization</li>
<li>Isolating tenants with quotas and rate limits</li>
<li>Monitoring and metering</li>
<li>Inter-cluster data sharing (cf. geo-replication)</li>
</ul>
<h4class="anchor-heading"><aid="multitenancy-topic-naming"class="anchor-link"></a><ahref="#multitenancy-topic-naming">Creating User Spaces (Namespaces) For Tenants With Topic Naming</a></h4>
<p>
Kafka administrators operating a multi-tenant cluster typically need to define user spaces for each tenant. For the purpose of this section, "user spaces" are a collection of topics, which are grouped together under the management of a single entity or user.
</p>
<p>
In Kafka, the main unit of data is the topic. Users can create and name each topic. They can also delete them, but it is not possible to rename a topic directly. Instead, to rename a topic, the user must create a new topic, move the messages from the original topic to the new, and then delete the original. With this in mind, it is recommended to define logical spaces, based on an hierarchical topic naming structure. This setup can then be combined with security features, such as prefixed ACLs, to isolate different spaces and tenants, while also minimizing the administrative overhead for securing the data in the cluster.
</p>
<p>
These logical user spaces can be grouped in different ways, and the concrete choice depends on how your organization prefers to use your Kafka clusters. The most common groupings are as follows.
</p>
<p>
<em>By team or organizational unit:</em> Here, the team is the main aggregator. In an organization where teams are the main user of the Kafka infrastructure, this might be the best grouping.
<em>By project or product:</em> Here, a team manages more than one project. Their credentials will be different for each project, so all the controls and settings will always be project related.
Certain information should normally not be put in a topic name, such as information that is likely to change over time (e.g., the name of the intended consumer) or that is a technical detail or metadata that is available elsewhere (e.g., the topic's partition count and other configuration settings).
</p>
<p>
To enforce a topic naming structure, several options are available:
</p>
<ul>
<li>Use <ahref="#security_authz">prefix ACLs</a> (cf. <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-290%3A+Support+for+Prefixed+ACLs">KIP-290</a>) to enforce a common prefix for topic names. For example, team A may only be permitted to create topics whose names start with <code>payments.teamA.</code>.</li>
<li>Define a custom <code>CreateTopicPolicy</code> (cf. <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-108%3A+Create+Topic+Policy">KIP-108</a> and the setting <ahref="#brokerconfigs_create.topic.policy.class.name">create.topic.policy.class.name</a>) to enforce strict naming patterns. These policies provide the most flexibility and can cover complex patterns and rules to match an organization's needs.</li>
<li>Disable topic creation for normal users by denying it with an ACL, and then rely on an external process to create topics on behalf of users (e.g., scripting or your favorite automation toolkit).</li>
<li>It may also be useful to disable the Kafka feature to auto-create topics on demand by setting <code>auto.create.topics.enable=false</code> in the broker configuration. Note that you should not rely solely on this option.</li>
</ul>
<h4class="anchor-heading"><aid="multitenancy-topic-configs"class="anchor-link"></a><ahref="#multitenancy-topic-configs">Configuring Topics: Data Retention And More</a></h4>
<p>
Kafka's configuration is very flexible due to its fine granularity, and it supports a plethora of <ahref="#topicconfigs">per-topic configuration settings</a> to help administrators set up multi-tenant clusters. For example, administrators often need to define data retention policies to control how much and/or for how long data will be stored in a topic, with settings such as <ahref="#retention.bytes">retention.bytes</a> (size) and <ahref="#retention.ms">retention.ms</a> (time). This limits storage consumption within the cluster, and helps complying with legal requirements such as GDPR.
</p>
<h4class="anchor-heading"><aid="multitenancy-security"class="anchor-link"></a><ahref="#multitenancy-security">Securing Clusters and Topics: Authentication, Authorization, Encryption</a></h4>
<p>
Because the documentation has a dedicated chapter on <ahref="#security">security</a> that applies to any Kafka deployment, this section focuses on additional considerations for multi-tenant environments.
</p>
<p>
Security settings for Kafka fall into three main categories, which are similar to how administrators would secure other client-server data systems, like relational databases and traditional messaging systems.
</p>
<ol>
<li><strong>Encryption</strong> of data transferred between Kafka brokers and Kafka clients, between brokers, between brokers and ZooKeeper nodes, and between brokers and other, optional tools.</li>
<li><strong>Authentication</strong> of connections from Kafka clients and applications to Kafka brokers, as well as connections from Kafka brokers to ZooKeeper nodes.</li>
<li><strong>Authorization</strong> of client operations such as creating, deleting, and altering the configuration of topics; writing events to or reading events from a topic; creating and deleting ACLs. Administrators can also define custom policies to put in place additional restrictions, such as a <code>CreateTopicPolicy</code> and <code>AlterConfigPolicy</code> (see <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-108%3A+Create+Topic+Policy">KIP-108</a> and the settings <ahref="#brokerconfigs_create.topic.policy.class.name">create.topic.policy.class.name</a>, <ahref="#brokerconfigs_alter.config.policy.class.name">alter.config.policy.class.name</a>).</li>
</ol>
<p>
When securing a multi-tenant Kafka environment, the most common administrative task is the third category (authorization), i.e., managing the user/client permissions that grant or deny access to certain topics and thus to the data stored by users within a cluster. This task is performed predominantly through the <ahref="#security_authz">setting of access control lists (ACLs)</a>. Here, administrators of multi-tenant environments in particular benefit from putting a hierarchical topic naming structure in place as described in a previous section, because they can conveniently control access to topics through prefixed ACLs (<code>--resource-pattern-type Prefixed</code>). This significantly minimizes the administrative overhead of securing topics in multi-tenant environments: administrators can make their own trade-offs between higher developer convenience (more lenient permissions, using fewer and broader ACLs) vs. tighter security (more stringent permissions, using more and narrower ACLs).
</p>
<p>
In the following example, user Alice—a new member of ACME corporation's InfoSec team—is granted write permissions to all topics whose names start with "acme.infosec.", such as "acme.infosec.telemetry.logins" and "acme.infosec.syslogs.events".
</p>
<preclass="line-numbers"><codeclass="language-text"># Grant permissions to user Alice
Multi-tenant clusters should generally be configured with <ahref="#design_quotas">quotas</a>, which protect against users (tenants) eating up too many cluster resources, such as when they attempt to write or read very high volumes of data, or create requests to brokers at an excessively high rate. This may cause network saturation, monopolize broker resources, and impact other clients—all of which you want to avoid in a shared environment.
</p>
<p>
<strong>Client quotas:</strong> Kafka supports different types of (per-user principal) client quotas. Because a client's quotas apply irrespective of which topics the client is writing to or reading from, they are a convenient and effective tool to allocate resources in a multi-tenant cluster. <ahref="#design_quotascpu">Request rate quotas</a>, for example, help to limit a user's impact on broker CPU usage by limiting the time a broker spends on the <ahref="/protocol.html">request handling path</a> for that user, after which throttling kicks in. In many situations, isolating users with request rate quotas has a bigger impact in multi-tenant clusters than setting incoming/outgoing network bandwidth quotas, because excessive broker CPU usage for processing requests reduces the effective bandwidth the broker can serve. Furthermore, administrators can also define quotas on topic operations—such as create, delete, and alter—to prevent Kafka clusters from being overwhelmed by highly concurrent topic operations (see <ahref="https://cwiki.apache.org/confluence/display/KAFKA/KIP-599%3A+Throttle+Create+Topic%2C+Create+Partition+and+Delete+Topic+Operations">KIP-599</a> and the quota type <code>controller_mutations_rate</code>).
</p>
<p>
<strong>Server quotas:</strong> Kafka also supports different types of broker-side quotas. For example, administrators can set a limit on the rate with which the <ahref="#brokerconfigs_max.connection.creation.rate">broker accepts new connections</a>, set the <ahref="#brokerconfigs_max.connections">maximum number of connections per broker</a>, or set the maximum number of connections allowed <ahref="#brokerconfigs_max.connections.per.ip">from a specific IP address</a>.
</p>
<p>
For more information, please refer to the <ahref="#design_quotas">quota overview</a> and <ahref="#quotas">how to set quotas</a>.
</p>
<h4class="anchor-heading"><aid="multitenancy-monitoring"class="anchor-link"></a><ahref="#multitenancy-monitoring">Monitoring and Metering</a></h4>
<p>
<ahref="#monitoring">Monitoring</a> is a broader subject that is covered <ahref="#monitoring">elsewhere</a> in the documentation. Administrators of any Kafka environment, but especially multi-tenant ones, should set up monitoring according to these instructions. Kafka supports a wide range of metrics, such as the rate of failed authentication attempts, request latency, consumer lag, total number of consumer groups, metrics on the quotas described in the previous section, and many more.
</p>
<p>
For example, monitoring can be configured to track the size of topic-partitions (with the JMX metric <code>kafka.log.Log.Size.<TOPIC-NAME></code>), and thus the total size of data stored in a topic. You can then define alerts when tenants on shared clusters are getting close to using too much storage space.
</p>
<h4class="anchor-heading"><aid="multitenancy-georeplication"class="anchor-link"></a><ahref="#multitenancy-georeplication">Multi-Tenancy and Geo-Replication</a></h4>
<p>
Kafka lets you share data across different clusters, which may be located in different geographical regions, data centers, and so on. Apart from use cases such as disaster recovery, this functionality is useful when a multi-tenant setup requires inter-cluster data sharing. See the section <ahref="#georeplication">Geo-Replication (Cross-Cluster Data Mirroring)</a> for more information.
<strong>Data contracts:</strong> You may need to define data contracts between the producers and the consumers of data in a cluster, using event schemas. This ensures that events written to Kafka can always be read properly again, and prevents malformed or corrupt events being written. The best way to achieve this is to deploy a so-called schema registry alongside the cluster. (Kafka does not include a schema registry, but there are third-party implementations available.) A schema registry manages the event schemas and maps the schemas to topics, so that producers know which topics are accepting which types (schemas) of events, and consumers know how to read and parse events in a topic. Some registry implementations provide further functionality, such as schema evolution, storing a history of all schemas, and schema compatibility settings.
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.
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).
<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 <ahref="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). <ahref="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.
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.
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.
In Linux, data written to the filesystem is maintained in <ahref="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 <ahref="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
<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>
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>
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>
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.
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.
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.
<h4class="anchor-heading"><aid="remote_jmx"class="anchor-link"></a><ahref="#remote_jmx">Security Considerations for Remote Monitoring using JMX</a></h4>
<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>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>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>
<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
<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>
<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>
<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>
<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>
<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>The average end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
<td>The maximum end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
<td>The minimum end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
<h5class="anchor-heading"><aid="kafka_streams_store_monitoring"class="anchor-link"></a><ahref="#kafka_streams_store_monitoring">State Store Metrics</a></h5>
All of the following metrics have a recording level of <code>debug</code>, except for the record-e2e-latency-* metrics which have a recording level <code>trace></code>.
Note that the <code>store-scope</code> value is specified in <code>StoreSupplier#metricsScope()</code> for user's customized state stores;
<td>The average end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
<td>The maximum end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
<td>The minimum end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
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.
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.