This is an automated email from the ASF dual-hosted git repository.
bbejeck pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/kafka-site.git
The following commit(s) were added to refs/heads/asf-site by this push:
new 901df6c Port changes for MM2 docs (KAFKA-8930) to 2.6 docs (#326)
901df6c is described below
commit 901df6c243ae406c6250a8840ca8fe071a57a18d
Author: Bill Bejeck <[email protected]>
AuthorDate: Wed Jan 27 13:25:01 2021 -0500
Port changes for MM2 docs (KAFKA-8930) to 2.6 docs (#326)
---
26/ops.html | 594 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++---
26/toc.html | 20 +-
2 files changed, 583 insertions(+), 31 deletions(-)
diff --git a/26/ops.html b/26/ops.html
index e835341..eb64156 100644
--- a/26/ops.html
+++ b/26/ops.html
@@ -85,32 +85,18 @@
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"> broker.rack=my-rack-id</code></pre>
+ <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</a></h4>
+ <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>
- We refer to the process of replicating data <i>between</i> Kafka clusters
"mirroring" to avoid confusion with the replication that happens amongst the
nodes in a single cluster. Kafka comes with a tool for mirroring data between
Kafka clusters. The tool consumes from a source cluster and produces to a
destination cluster.
-
- A common use case for this kind of mirroring is to provide a replica in
another datacenter. This scenario will be discussed in more detail in the next
section.
- <p>
- You can run many such mirroring processes to increase throughput and for
fault-tolerance (if one process dies, the others will take overs the additional
load).
- <p>
- Data will be read from topics in the source cluster and written to a topic
with the same name in the destination cluster. In fact the mirror maker is
little more than a Kafka consumer and producer hooked together.
<p>
- The source and destination clusters are completely independent entities:
they can have different numbers of partitions and the offsets will not be the
same. For this reason the mirror cluster is not really intended as a
fault-tolerance mechanism (as the consumer position will be different); for
that we recommend using normal in-cluster replication. The mirror maker process
will, however, retain and use the message key for partitioning so order is
preserved on a per-key basis.
- <p>
- Here is an example showing how to mirror a single topic (named
<i>my-topic</i>) from an input cluster:
- <pre class="line-numbers"><code class="language-bash"> >
bin/kafka-mirror-maker.sh
- --consumer.config consumer.properties
- --producer.config producer.properties --whitelist my-topic</code></pre>
- Note that we specify the list of topics with the <code>--whitelist</code>
option. This option allows any regular expression using <a
href="http://docs.oracle.com/javase/7/docs/api/java/util/regex/Pattern.html">Java-style
regular expressions</a>. So you could mirror two topics named <i>A</i> and
<i>B</i> using <code>--whitelist 'A|B'</code>. Or you could mirror <i>all</i>
topics using <code>--whitelist '*'</code>. Make sure to quote any regular
expression to ensure the shell doesn't try [...]
-
- Combining mirroring with the configuration
<code>auto.create.topics.enable=true</code> makes it possible to have a replica
cluster that will automatically create and replicate all data in a source
cluster even as new topics are added.
+ 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:
@@ -472,12 +458,12 @@
<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><code>max(BytesInPerSec) > throttle</code></pre>
+ <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><code>kafka.server:type=FetcherLagMetrics,name=ConsumerLag,clientId=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)</code></pre>
+
<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
@@ -541,7 +527,7 @@
<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 between clusters (see
the documentation on the <a href="#basic_ops_mirror_maker">mirror maker
tool</a> for how to do this).
+ 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>
@@ -553,7 +539,558 @@
<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="config" class="anchor-link"></a><a
href="#config">6.3 Kafka Configuration</a></h3>
+ <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 t [...]
+ </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 [...]
+ </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 replica [...]
+ </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 [...]
+ </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 <MirrorMaker
pid>
+</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>
@@ -586,7 +1123,7 @@
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.4 Java Version</a></h3>
+ <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).
@@ -609,7 +1146,7 @@
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.5 Hardware and OS</a></h3>
+ <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.
@@ -694,7 +1231,7 @@
<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.6 Monitoring</a></h3>
+ <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>
@@ -767,6 +1304,11 @@
<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>
@@ -2143,7 +2685,7 @@ dropped-records-rate and dropped-records-total which have
a recording level of <
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.7 ZooKeeper</a></h3>
+ <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.
diff --git a/26/toc.html b/26/toc.html
index b8c977a..8d15b2b 100644
--- a/26/toc.html
+++ b/26/toc.html
@@ -86,13 +86,23 @@
<li><a
href="#basic_ops_increase_replication_factor">Increasing replication factor</a>
</ul>
<li><a href="#datacenters">6.2 Datacenters</a>
- <li><a href="#config">6.3 Important Configs</a>
+ <li><a href="#georeplication">6.3 Geo-Replication
(Cross-Cluster Data Mirroring)</a></li>
+ <ul>
+ <li><a href="#georeplication-overview">Geo-Replication
Overview</a></li>
+ <li><a href="#georeplication-flows">What Are
Replication Flows</a></li>
+ <li><a href="#georeplication-mirrormaker">Configuring
Geo-Replication</a></li>
+ <li><a href="#georeplication-starting">Starting
Geo-Replication</a></li>
+ <li><a href="#georeplication-stopping">Stopping
Geo-Replication</a></li>
+ <li><a
href="#georeplication-apply-config-changes">Applying Configuration
Changes</a></li>
+ <li><a href="#georeplication-monitoring">Monitoring
Geo-Replication</a></li>
+ </ul>
+ <li><a href="#config">6.4 Important Configs</a>
<ul>
<li><a href="#clientconfig">Important Client
Configs</a>
<li><a href="#prodconfig">A Production Server
Configs</a>
</ul>
- <li><a href="#java">6.4 Java Version</a>
- <li><a href="#hwandos">6.5 Hardware and OS</a>
+ <li><a href="#java">6.5 Java Version</a>
+ <li><a href="#hwandos">6.6 Hardware and OS</a>
<ul>
<li><a href="#os">OS</a>
<li><a href="#diskandfs">Disks and Filesystems</a>
@@ -100,7 +110,7 @@
<li><a href="#linuxflush">Linux Flush Behavior</a>
<li><a href="#ext4">Ext4 Notes</a>
</ul>
- <li><a href="#monitoring">6.6 Monitoring</a>
+ <li><a href="#monitoring">6.7 Monitoring</a>
<ul>
<li><a href="#selector_monitoring">Selector
Monitoring</a></li>
<li><a href="#common_node_monitoring">Common Node
Monitoring</a></li>
@@ -110,7 +120,7 @@
<li><a href="#kafka_streams_monitoring">Streams
Monitoring</a></li>
<li><a href="#others_monitoring">Others</a></li>
</ul>
- <li><a href="#zk">6.7 ZooKeeper</a>
+ <li><a href="#zk">6.8 ZooKeeper</a>
<ul>
<li><a href="#zkversion">Stable Version</a>
<li><a href="#zkops">Operationalization</a>