forked from apache/kafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathops.html
4094 lines (3613 loc) · 243 KB
/
ops.html
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
Here is some information on actually running Kafka as a production system based on usage and experience at LinkedIn. Please send us any additional tips you know of.
<h3 class="anchor-heading"><a id="basic_ops" class="anchor-link"></a><a href="#basic_ops">6.1 Basic Kafka Operations</a></h3>
This section will review the most common operations you will perform on your Kafka cluster. All of the tools reviewed in this section are available under the <code>bin/</code> directory of the Kafka distribution and each tool will print details on all possible commandline options if it is run with no arguments.
<h4 class="anchor-heading"><a id="basic_ops_add_topic" class="anchor-link"></a><a href="#basic_ops_add_topic">Adding and removing topics</a></h4>
You have the option of either adding topics manually or having them be created automatically when data is first published to a non-existent topic. If topics are auto-created then you may want to tune the default <a href="#topicconfigs">topic configurations</a> used for auto-created topics.
<p>
Topics are added and modified using the topic tool:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-topics.sh --bootstrap-server broker_host:port --create --topic my_topic_name \
--partitions 20 --replication-factor 3 --config x=y</code></pre>
The replication factor controls how many servers will replicate each message that is written. If you have a replication factor of 3 then up to 2 servers can fail before you will lose access to your data. We recommend you use a replication factor of 2 or 3 so that you can transparently bounce machines without interrupting data consumption.
<p>
The partition count controls how many logs the topic will be sharded into. There are several impacts of the partition count. First each partition must fit entirely on a single server. So if you have 20 partitions the full data set (and read and write load) will be handled by no more than 20 servers (not counting replicas). Finally the partition count impacts the maximum parallelism of your consumers. This is discussed in greater detail in the <a href="#intro_consumers">concepts section</a>.
<p>
Each sharded partition log is placed into its own folder under the Kafka log directory. The name of such folders consists of the topic name, appended by a dash (-) and the partition id. Since a typical folder name can not be over 255 characters long, there will be a limitation on the length of topic names. We assume the number of partitions will not ever be above 100,000. Therefore, topic names cannot be longer than 249 characters. This leaves just enough room in the folder name for a dash and a potentially 5 digit long partition id.
<p>
The configurations added on the command line override the default settings the server has for things like the length of time data should be retained. The complete set of per-topic configurations is documented <a href="#topicconfigs">here</a>.
<h4 class="anchor-heading"><a id="basic_ops_modify_topic" class="anchor-link"></a><a href="#basic_ops_modify_topic">Modifying topics</a></h4>
You can change the configuration or partitioning of a topic using the same topic tool.
<p>
To add partitions you can do
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-topics.sh --bootstrap-server broker_host:port --alter --topic my_topic_name \
--partitions 40</code></pre>
Be aware that one use case for partitions is to semantically partition data, and adding partitions doesn't change the partitioning of existing data so this may disturb consumers if they rely on that partition. That is if data is partitioned by <code>hash(key) % number_of_partitions</code> then this partitioning will potentially be shuffled by adding partitions but Kafka will not attempt to automatically redistribute data in any way.
<p>
To add configs:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server broker_host:port --entity-type topics --entity-name my_topic_name --alter --add-config x=y</code></pre>
To remove a config:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server broker_host:port --entity-type topics --entity-name my_topic_name --alter --delete-config x</code></pre>
And finally deleting a topic:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-topics.sh --bootstrap-server broker_host:port --delete --topic my_topic_name</code></pre>
<p>
Kafka does not currently support reducing the number of partitions for a topic.
<p>
Instructions for changing the replication factor of a topic can be found <a href="#basic_ops_increase_replication_factor">here</a>.
<h4 class="anchor-heading"><a id="basic_ops_restarting" class="anchor-link"></a><a href="#basic_ops_restarting">Graceful shutdown</a></h4>
The Kafka cluster will automatically detect any broker shutdown or failure and elect new leaders for the partitions on that machine. This will occur whether a server fails or it is brought down intentionally for maintenance or configuration changes. For the latter cases Kafka supports a more graceful mechanism for stopping a server than just killing it.
When a server is stopped gracefully it has two optimizations it will take advantage of:
<ol>
<li>It will sync all its logs to disk to avoid needing to do any log recovery when it restarts (i.e. validating the checksum for all messages in the tail of the log). Log recovery takes time so this speeds up intentional restarts.
<li>It will migrate any partitions the server is the leader for to other replicas prior to shutting down. This will make the leadership transfer faster and minimize the time each partition is unavailable to a few milliseconds.
</ol>
Syncing the logs will happen automatically whenever the server is stopped other than by a hard kill, but the controlled leadership migration requires using a special setting:
<pre class="line-numbers"><code class="language-text"> controlled.shutdown.enable=true</code></pre>
Note that controlled shutdown will only succeed if <i>all</i> the partitions hosted on the broker have replicas (i.e. the replication factor is greater than 1 <i>and</i> at least one of these replicas is alive). This is generally what you want since shutting down the last replica would make that topic partition unavailable.
<h4 class="anchor-heading"><a id="basic_ops_leader_balancing" class="anchor-link"></a><a href="#basic_ops_leader_balancing">Balancing leadership</a></h4>
Whenever a broker stops or crashes, leadership for that broker's partitions transfers to other replicas. When the broker is restarted it will only be a follower for all its partitions, meaning it will not be used for client reads and writes.
<p>
To avoid this imbalance, Kafka has a notion of preferred replicas. If the list of replicas for a partition is 1,5,9 then node 1 is preferred as the leader to either node 5 or 9 because it is earlier in the replica list. By default the Kafka cluster will try to restore leadership to the preferred replicas. This behaviour is configured with:
<pre class="line-numbers"><code class="language-text"> auto.leader.rebalance.enable=true</code></pre>
You can also set this to false, but you will then need to manually restore leadership to the restored replicas by running the command:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-leader-election.sh --bootstrap-server broker_host:port --election-type preferred --all-topic-partitions</code></pre>
<h4 class="anchor-heading"><a id="basic_ops_racks" class="anchor-link"></a><a href="#basic_ops_racks">Balancing Replicas Across Racks</a></h4>
The rack awareness feature spreads replicas of the same partition across different racks. This extends the guarantees Kafka provides for broker-failure to cover rack-failure, limiting the risk of data loss should all the brokers on a rack fail at once. The feature can also be applied to other broker groupings such as availability zones in EC2.
<p></p>
You can specify that a broker belongs to a particular rack by adding a property to the broker config:
<pre class="language-text"><code class="language-text"> broker.rack=my-rack-id</code></pre>
When a topic is <a href="#basic_ops_add_topic">created</a>, <a href="#basic_ops_modify_topic">modified</a> or replicas are <a href="#basic_ops_cluster_expansion">redistributed</a>, the rack constraint will be honoured, ensuring replicas span as many racks as they can (a partition will span min(#racks, replication-factor) different racks).
<p></p>
The algorithm used to assign replicas to brokers ensures that the number of leaders per broker will be constant, regardless of how brokers are distributed across racks. This ensures balanced throughput.
<p></p>
However if racks are assigned different numbers of brokers, the assignment of replicas will not be even. Racks with fewer brokers will get more replicas, meaning they will use more storage and put more resources into replication. Hence it is sensible to configure an equal number of brokers per rack.
<h4 class="anchor-heading"><a id="basic_ops_mirror_maker" class="anchor-link"></a><a href="#basic_ops_mirror_maker">Mirroring data between clusters & Geo-replication</a></h4>
<p>
Kafka administrators can define data flows that cross the boundaries of individual Kafka clusters, data centers, or geographical regions. Please refer to the section on <a href="#georeplication">Geo-Replication</a> for further information.
</p>
<h4 class="anchor-heading"><a id="basic_ops_consumer_lag" class="anchor-link"></a><a href="#basic_ops_consumer_lag">Checking consumer position</a></h4>
Sometimes it's useful to see the position of your consumers. We have a tool that will show the position of all consumers in a consumer group as well as how far behind the end of the log they are. To run this tool on a consumer group named <i>my-group</i> consuming a topic named <i>my-topic</i> would look like this:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --describe --group my-group
TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID HOST CLIENT-ID
my-topic 0 2 4 2 consumer-1-029af89c-873c-4751-a720-cefd41a669d6 /127.0.0.1 consumer-1
my-topic 1 2 3 1 consumer-1-029af89c-873c-4751-a720-cefd41a669d6 /127.0.0.1 consumer-1
my-topic 2 2 3 1 consumer-2-42c1abd4-e3b2-425d-a8bb-e1ea49b29bb2 /127.0.0.1 consumer-2</code></pre>
<h4 class="anchor-heading"><a id="basic_ops_consumer_group" class="anchor-link"></a><a href="#basic_ops_consumer_group">Managing Consumer Groups</a></h4>
With the ConsumerGroupCommand tool, we can list, describe, or delete the consumer groups. The consumer group can be deleted manually, or automatically when the last committed offset for that group expires. Manual deletion works only if the group does not have any active members.
For example, to list all consumer groups across all topics:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --list
test-consumer-group</code></pre>
To view offsets, as mentioned earlier, we "describe" the consumer group like this:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --describe --group my-group
TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID HOST CLIENT-ID
topic3 0 241019 395308 154289 consumer2-e76ea8c3-5d30-4299-9005-47eb41f3d3c4 /127.0.0.1 consumer2
topic2 1 520678 803288 282610 consumer2-e76ea8c3-5d30-4299-9005-47eb41f3d3c4 /127.0.0.1 consumer2
topic3 1 241018 398817 157799 consumer2-e76ea8c3-5d30-4299-9005-47eb41f3d3c4 /127.0.0.1 consumer2
topic1 0 854144 855809 1665 consumer1-3fc8d6f1-581a-4472-bdf3-3515b4aee8c1 /127.0.0.1 consumer1
topic2 0 460537 803290 342753 consumer1-3fc8d6f1-581a-4472-bdf3-3515b4aee8c1 /127.0.0.1 consumer1
topic3 2 243655 398812 155157 consumer4-117fe4d3-c6c1-4178-8ee9-eb4a3954bee0 /127.0.0.1 consumer4</code></pre>
There are a number of additional "describe" options that can be used to provide more detailed information about a consumer group:
<ul>
<li>--members: This option provides the list of all active members in the consumer group.
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --describe --group my-group --members
CONSUMER-ID HOST CLIENT-ID #PARTITIONS
consumer1-3fc8d6f1-581a-4472-bdf3-3515b4aee8c1 /127.0.0.1 consumer1 2
consumer4-117fe4d3-c6c1-4178-8ee9-eb4a3954bee0 /127.0.0.1 consumer4 1
consumer2-e76ea8c3-5d30-4299-9005-47eb41f3d3c4 /127.0.0.1 consumer2 3
consumer3-ecea43e4-1f01-479f-8349-f9130b75d8ee /127.0.0.1 consumer3 0</code></pre>
</li>
<li>--members --verbose: On top of the information reported by the "--members" options above, this option also provides the partitions assigned to each member.
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --describe --group my-group --members --verbose
CONSUMER-ID HOST CLIENT-ID #PARTITIONS ASSIGNMENT
consumer1-3fc8d6f1-581a-4472-bdf3-3515b4aee8c1 /127.0.0.1 consumer1 2 topic1(0), topic2(0)
consumer4-117fe4d3-c6c1-4178-8ee9-eb4a3954bee0 /127.0.0.1 consumer4 1 topic3(2)
consumer2-e76ea8c3-5d30-4299-9005-47eb41f3d3c4 /127.0.0.1 consumer2 3 topic2(1), topic3(0,1)
consumer3-ecea43e4-1f01-479f-8349-f9130b75d8ee /127.0.0.1 consumer3 0 -</code></pre>
</li>
<li>--offsets: This is the default describe option and provides the same output as the "--describe" option.</li>
<li>--state: This option provides useful group-level information.
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --describe --group my-group --state
COORDINATOR (ID) ASSIGNMENT-STRATEGY STATE #MEMBERS
localhost:9092 (0) range Stable 4</code></pre>
</li>
</ul>
To manually delete one or multiple consumer groups, the "--delete" option can be used:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --delete --group my-group --group my-other-group
Deletion of requested consumer groups ('my-group', 'my-other-group') was successful.</code></pre>
<p>
To reset offsets of a consumer group, "--reset-offsets" option can be used.
This option supports one consumer group at the time. It requires defining following scopes: --all-topics or --topic. One scope must be selected, unless you use '--from-file' scenario. Also, first make sure that the consumer instances are inactive.
See <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-122%3A+Add+Reset+Consumer+Group+Offsets+tooling">KIP-122</a> for more details.
<p>
It has 3 execution options:
<ul>
<li>
(default) to display which offsets to reset.
</li>
<li>
--execute : to execute --reset-offsets process.
</li>
<li>
--export : to export the results to a CSV format.
</li>
</ul>
<p>
--reset-offsets also has following scenarios to choose from (at least one scenario must be selected):
<ul>
<li>
--to-datetime <String: datetime> : Reset offsets to offsets from datetime. Format: 'YYYY-MM-DDTHH:mm:SS.sss'
</li>
<li>
--to-earliest : Reset offsets to earliest offset.
</li>
<li>
--to-latest : Reset offsets to latest offset.
</li>
<li>
--shift-by <Long: number-of-offsets> : Reset offsets shifting current offset by 'n', where 'n' can be positive or negative.
</li>
<li>
--from-file : Reset offsets to values defined in CSV file.
</li>
<li>
--to-current : Resets offsets to current offset.
</li>
<li>
--by-duration <String: duration> : Reset offsets to offset by duration from current timestamp. Format: 'PnDTnHnMnS'
</li>
<li>
--to-offset : Reset offsets to a specific offset.
</li>
</ul>
Please note, that out of range offsets will be adjusted to available offset end. For example, if offset end is at 10 and offset shift request is
of 15, then, offset at 10 will actually be selected.
<p>
For example, to reset offsets of a consumer group to the latest offset:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --reset-offsets --group consumergroup1 --topic topic1 --to-latest
TOPIC PARTITION NEW-OFFSET
topic1 0 0</code></pre>
<p>
If you are using the old high-level consumer and storing the group metadata in ZooKeeper (i.e. <code>offsets.storage=zookeeper</code>), pass
<code>--zookeeper</code> instead of <code>--bootstrap-server</code>:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-consumer-groups.sh --zookeeper localhost:2181 --list</code></pre>
<h4 class="anchor-heading"><a id="basic_ops_cluster_expansion" class="anchor-link"></a><a href="#basic_ops_cluster_expansion">Expanding your cluster</a></h4>
Adding servers to a Kafka cluster is easy, just assign them a unique broker id and start up Kafka on your new servers. However these new servers will not automatically be assigned any data partitions, so unless partitions are moved to them they won't be doing any work until new topics are created. So usually when you add machines to your cluster you will want to migrate some existing data to these machines.
<p>
The process of migrating data is manually initiated but fully automated. Under the covers what happens is that Kafka will add the new server as a follower of the partition it is migrating and allow it to fully replicate the existing data in that partition. When the new server has fully replicated the contents of this partition and joined the in-sync replica one of the existing replicas will delete their partition's data.
<p>
The partition reassignment tool can be used to move partitions across brokers. An ideal partition distribution would ensure even data load and partition sizes across all brokers. The partition reassignment tool does not have the capability to automatically study the data distribution in a Kafka cluster and move partitions around to attain an even load distribution. As such, the admin has to figure out which topics or partitions should be moved around.
<p>
The partition reassignment tool can run in 3 mutually exclusive modes:
<ul>
<li>--generate: In this mode, given a list of topics and a list of brokers, the tool generates a candidate reassignment to move all partitions of the specified topics to the new brokers. This option merely provides a convenient way to generate a partition reassignment plan given a list of topics and target brokers.</li>
<li>--execute: In this mode, the tool kicks off the reassignment of partitions based on the user provided reassignment plan. (using the --reassignment-json-file option). This can either be a custom reassignment plan hand crafted by the admin or provided by using the --generate option</li>
<li>--verify: In this mode, the tool verifies the status of the reassignment for all partitions listed during the last --execute. The status can be either of successfully completed, failed or in progress</li>
</ul>
<h5 class="anchor-heading"><a id="basic_ops_automigrate" class="anchor-link"></a><a href="#basic_ops_automigrate">Automatically migrating data to new machines</a></h5>
The partition reassignment tool can be used to move some topics off of the current set of brokers to the newly added brokers. This is typically useful while expanding an existing cluster since it is easier to move entire topics to the new set of brokers, than moving one partition at a time. When used to do this, the user should provide a list of topics that should be moved to the new set of brokers and a target list of new brokers. The tool then evenly distributes all partitions for the given list of topics across the new set of brokers. During this move, the replication factor of the topic is kept constant. Effectively the replicas for all partitions for the input list of topics are moved from the old set of brokers to the newly added brokers.
<p>
For instance, the following example will move all partitions for topics foo1,foo2 to the new set of brokers 5,6. At the end of this move, all partitions for topics foo1 and foo2 will <i>only</i> exist on brokers 5,6.
<p>
Since the tool accepts the input list of topics as a json file, you first need to identify the topics you want to move and create the json file as follows:
<pre class="line-numbers"><code class="language-bash"> > cat topics-to-move.json
{"topics": [{"topic": "foo1"},
{"topic": "foo2"}],
"version":1
}</code></pre>
Once the json file is ready, use the partition reassignment tool to generate a candidate assignment:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --topics-to-move-json-file topics-to-move.json --broker-list "5,6" --generate
Current partition replica assignment
{"version":1,
"partitions":[{"topic":"foo1","partition":0,"replicas":[2,1]},
{"topic":"foo1","partition":1,"replicas":[1,3]},
{"topic":"foo1","partition":2,"replicas":[3,4]},
{"topic":"foo2","partition":0,"replicas":[4,2]},
{"topic":"foo2","partition":1,"replicas":[2,1]},
{"topic":"foo2","partition":2,"replicas":[1,3]}]
}
Proposed partition reassignment configuration
{"version":1,
"partitions":[{"topic":"foo1","partition":0,"replicas":[6,5]},
{"topic":"foo1","partition":1,"replicas":[5,6]},
{"topic":"foo1","partition":2,"replicas":[6,5]},
{"topic":"foo2","partition":0,"replicas":[5,6]},
{"topic":"foo2","partition":1,"replicas":[6,5]},
{"topic":"foo2","partition":2,"replicas":[5,6]}]
}</code></pre>
<p>
The tool generates a candidate assignment that will move all partitions from topics foo1,foo2 to brokers 5,6. Note, however, that at this point, the partition movement has not started, it merely tells you the current assignment and the proposed new assignment. The current assignment should be saved in case you want to rollback to it. The new assignment should be saved in a json file (e.g. expand-cluster-reassignment.json) to be input to the tool with the --execute option as follows:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file expand-cluster-reassignment.json --execute
Current partition replica assignment
{"version":1,
"partitions":[{"topic":"foo1","partition":0,"replicas":[2,1]},
{"topic":"foo1","partition":1,"replicas":[1,3]},
{"topic":"foo1","partition":2,"replicas":[3,4]},
{"topic":"foo2","partition":0,"replicas":[4,2]},
{"topic":"foo2","partition":1,"replicas":[2,1]},
{"topic":"foo2","partition":2,"replicas":[1,3]}]
}
Save this to use as the --reassignment-json-file option during rollback
Successfully started partition reassignments for foo1-0,foo1-1,foo1-2,foo2-0,foo2-1,foo2-2
</code></pre>
<p>
Finally, the --verify option can be used with the tool to check the status of the partition reassignment. Note that the same expand-cluster-reassignment.json (used with the --execute option) should be used with the --verify option:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file expand-cluster-reassignment.json --verify
Status of partition reassignment:
Reassignment of partition [foo1,0] is completed
Reassignment of partition [foo1,1] is still in progress
Reassignment of partition [foo1,2] is still in progress
Reassignment of partition [foo2,0] is completed
Reassignment of partition [foo2,1] is completed
Reassignment of partition [foo2,2] is completed</code></pre>
<h5 class="anchor-heading"><a id="basic_ops_partitionassignment" class="anchor-link"></a><a href="#basic_ops_partitionassignment">Custom partition assignment and migration</a></h5>
The partition reassignment tool can also be used to selectively move replicas of a partition to a specific set of brokers. When used in this manner, it is assumed that the user knows the reassignment plan and does not require the tool to generate a candidate reassignment, effectively skipping the --generate step and moving straight to the --execute step
<p>
For instance, the following example moves partition 0 of topic foo1 to brokers 5,6 and partition 1 of topic foo2 to brokers 2,3:
<p>
The first step is to hand craft the custom reassignment plan in a json file:
<pre class="line-numbers"><code class="language-bash"> > cat custom-reassignment.json
{"version":1,"partitions":[{"topic":"foo1","partition":0,"replicas":[5,6]},{"topic":"foo2","partition":1,"replicas":[2,3]}]}</code></pre>
Then, use the json file with the --execute option to start the reassignment process:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file custom-reassignment.json --execute
Current partition replica assignment
{"version":1,
"partitions":[{"topic":"foo1","partition":0,"replicas":[1,2]},
{"topic":"foo2","partition":1,"replicas":[3,4]}]
}
Save this to use as the --reassignment-json-file option during rollback
Successfully started partition reassignments for foo1-0,foo2-1
</code></pre>
<p>
The --verify option can be used with the tool to check the status of the partition reassignment. Note that the same custom-reassignment.json (used with the --execute option) should be used with the --verify option:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file custom-reassignment.json --verify
Status of partition reassignment:
Reassignment of partition [foo1,0] is completed
Reassignment of partition [foo2,1] is completed</code></pre>
<h4 class="anchor-heading"><a id="basic_ops_decommissioning_brokers" class="anchor-link"></a><a href="#basic_ops_decommissioning_brokers">Decommissioning brokers</a></h4>
The partition reassignment tool does not have the ability to automatically generate a reassignment plan for decommissioning brokers yet. As such, the admin has to come up with a reassignment plan to move the replica for all partitions hosted on the broker to be decommissioned, to the rest of the brokers. This can be relatively tedious as the reassignment needs to ensure that all the replicas are not moved from the decommissioned broker to only one other broker. To make this process effortless, we plan to add tooling support for decommissioning brokers in the future.
<h4 class="anchor-heading"><a id="basic_ops_increase_replication_factor" class="anchor-link"></a><a href="#basic_ops_increase_replication_factor">Increasing replication factor</a></h4>
Increasing the replication factor of an existing partition is easy. Just specify the extra replicas in the custom reassignment json file and use it with the --execute option to increase the replication factor of the specified partitions.
<p>
For instance, the following example increases the replication factor of partition 0 of topic foo from 1 to 3. Before increasing the replication factor, the partition's only replica existed on broker 5. As part of increasing the replication factor, we will add more replicas on brokers 6 and 7.
<p>
The first step is to hand craft the custom reassignment plan in a json file:
<pre class="line-numbers"><code class="language-bash"> > cat increase-replication-factor.json
{"version":1,
"partitions":[{"topic":"foo","partition":0,"replicas":[5,6,7]}]}</code></pre>
Then, use the json file with the --execute option to start the reassignment process:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file increase-replication-factor.json --execute
Current partition replica assignment
{"version":1,
"partitions":[{"topic":"foo","partition":0,"replicas":[5]}]}
Save this to use as the --reassignment-json-file option during rollback
Successfully started partition reassignment for foo-0</code></pre>
<p>
The --verify option can be used with the tool to check the status of the partition reassignment. Note that the same increase-replication-factor.json (used with the --execute option) should be used with the --verify option:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --reassignment-json-file increase-replication-factor.json --verify
Status of partition reassignment:
Reassignment of partition [foo,0] is completed</code></pre>
You can also verify the increase in replication factor with the kafka-topics tool:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-topics.sh --bootstrap-server localhost:9092 --topic foo --describe
Topic:foo PartitionCount:1 ReplicationFactor:3 Configs:
Topic: foo Partition: 0 Leader: 5 Replicas: 5,6,7 Isr: 5,6,7</code></pre>
<h4 class="anchor-heading"><a id="rep-throttle" class="anchor-link"></a><a href="#rep-throttle">Limiting Bandwidth Usage during Data Migration</a></h4>
Kafka lets you apply a throttle to replication traffic, setting an upper bound on the bandwidth used to move replicas from machine to machine. This is useful when rebalancing a cluster, bootstrapping a new broker or adding or removing brokers, as it limits the impact these data-intensive operations will have on users.
<p></p>
There are two interfaces that can be used to engage a throttle. The simplest, and safest, is to apply a throttle when invoking the kafka-reassign-partitions.sh, but kafka-configs.sh can also be used to view and alter the throttle values directly.
<p></p>
So for example, if you were to execute a rebalance, with the below command, it would move partitions at no more than 50MB/s.
<pre class="language-bash">$ bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --execute --reassignment-json-file bigger-cluster.json --throttle 50000000</code></pre>
When you execute this script you will see the throttle engage:
<pre class="line-numbers"><code class="language-bash"> The inter-broker throttle limit was set to 50000000 B/s
Successfully started partition reassignment for foo1-0</code></pre>
<p>Should you wish to alter the throttle, during a rebalance, say to increase the throughput so it completes quicker, you can do this by re-running the execute command with the --additional option passing the same reassignment-json-file:</p>
<pre class="language-bash">$ bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --additional --execute --reassignment-json-file bigger-cluster.json --throttle 700000000
The inter-broker throttle limit was set to 700000000 B/s</code></pre>
<p>Once the rebalance completes the administrator can check the status of the rebalance using the --verify option.
If the rebalance has completed, the throttle will be removed via the --verify command. It is important that
administrators remove the throttle in a timely manner once rebalancing completes by running the command with
the --verify option. Failure to do so could cause regular replication traffic to be throttled. </p>
<p>When the --verify option is executed, and the reassignment has completed, the script will confirm that the throttle was removed:</p>
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-reassign-partitions.sh --bootstrap-server localhost:9092 --verify --reassignment-json-file bigger-cluster.json
Status of partition reassignment:
Reassignment of partition [my-topic,1] is completed
Reassignment of partition [my-topic,0] is completed
Clearing broker-level throttles on brokers 1,2,3
Clearing topic-level throttles on topic my-topic</code></pre>
<p>The administrator can also validate the assigned configs using the kafka-configs.sh. There are two pairs of throttle
configuration used to manage the throttling process. First pair refers to the throttle value itself. This is configured, at a broker
level, using the dynamic properties: </p>
<pre class="line-numbers"><code class="language-text"> leader.replication.throttled.rate
follower.replication.throttled.rate</code></pre>
<p>Then there is the configuration pair of enumerated sets of throttled replicas: </p>
<pre class="line-numbers"><code class="language-text"> leader.replication.throttled.replicas
follower.replication.throttled.replicas</code></pre>
<p>Which are configured per topic. </p>
<p>All four config values are automatically assigned by kafka-reassign-partitions.sh (discussed below).</p>
<p>To view the throttle limit configuration:</p>
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --describe --bootstrap-server localhost:9092 --entity-type brokers
Configs for brokers '2' are leader.replication.throttled.rate=700000000,follower.replication.throttled.rate=700000000
Configs for brokers '1' are leader.replication.throttled.rate=700000000,follower.replication.throttled.rate=700000000</code></pre>
<p>This shows the throttle applied to both leader and follower side of the replication protocol. By default both sides
are assigned the same throttled throughput value. </p>
<p>To view the list of throttled replicas:</p>
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --describe --bootstrap-server localhost:9092 --entity-type topics
Configs for topic 'my-topic' are leader.replication.throttled.replicas=1:102,0:101,
follower.replication.throttled.replicas=1:101,0:102</code></pre>
<p>Here we see the leader throttle is applied to partition 1 on broker 102 and partition 0 on broker 101. Likewise the
follower throttle is applied to partition 1 on
broker 101 and partition 0 on broker 102. </p>
<p>By default kafka-reassign-partitions.sh will apply the leader throttle to all replicas that exist before the
rebalance, any one of which might be leader.
It will apply the follower throttle to all move destinations. So if there is a partition with replicas on brokers
101,102, being reassigned to 102,103, a leader throttle,
for that partition, would be applied to 101,102 and a follower throttle would be applied to 103 only. </p>
<p>If required, you can also use the --alter switch on kafka-configs.sh to alter the throttle configurations manually.
</p>
<h5>Safe usage of throttled replication</h5>
<p>Some care should be taken when using throttled replication. In particular:</p>
<p><i>(1) Throttle Removal:</i></p>
The throttle should be removed in a timely manner once reassignment completes (by running kafka-reassign-partitions.sh
--verify).
<p><i>(2) Ensuring Progress:</i></p>
<p>If the throttle is set too low, in comparison to the incoming write rate, it is possible for replication to not
make progress. This occurs when:</p>
<pre>max(BytesInPerSec) > throttle</code></pre>
<p>
Where BytesInPerSec is the metric that monitors the write throughput of producers into each broker. </p>
<p>The administrator can monitor whether replication is making progress, during the rebalance, using the metric:</p>
<pre>kafka.server:type=FetcherLagMetrics,name=ConsumerLag,clientId=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)</code></pre>
<p>The lag should constantly decrease during replication. If the metric does not decrease the administrator should
increase the
throttle throughput as described above. </p>
<h4 class="anchor-heading"><a id="quotas" class="anchor-link"></a><a href="#quotas">Setting quotas</a></h4>
Quotas overrides and defaults may be configured at (user, client-id), user or client-id levels as described <a href="#design_quotas">here</a>.
By default, clients receive an unlimited quota.
It is possible to set custom quotas for each (user, client-id), user or client-id group.
<p>
Configure custom quota for (user=user1, client-id=clientA):
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type users --entity-name user1 --entity-type clients --entity-name clientA
Updated config for entity: user-principal 'user1', client-id 'clientA'.</code></pre>
Configure custom quota for user=user1:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type users --entity-name user1
Updated config for entity: user-principal 'user1'.</code></pre>
Configure custom quota for client-id=clientA:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type clients --entity-name clientA
Updated config for entity: client-id 'clientA'.</code></pre>
It is possible to set default quotas for each (user, client-id), user or client-id group by specifying <i>--entity-default</i> option instead of <i>--entity-name</i>.
<p>
Configure default client-id quota for user=userA:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type users --entity-name user1 --entity-type clients --entity-default
Updated config for entity: user-principal 'user1', default client-id.</code></pre>
Configure default quota for user:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type users --entity-default
Updated config for entity: default user-principal.</code></pre>
Configure default quota for client-id:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200' --entity-type clients --entity-default
Updated config for entity: default client-id.</code></pre>
Here's how to describe the quota for a given (user, client-id):
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --describe --entity-type users --entity-name user1 --entity-type clients --entity-name clientA
Configs for user-principal 'user1', client-id 'clientA' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200</code></pre>
Describe quota for a given user:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --describe --entity-type users --entity-name user1
Configs for user-principal 'user1' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200</code></pre>
Describe quota for a given client-id:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --describe --entity-type clients --entity-name clientA
Configs for client-id 'clientA' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200</code></pre>
If entity name is not specified, all entities of the specified type are described. For example, describe all users:
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --describe --entity-type users
Configs for user-principal 'user1' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200
Configs for default user-principal are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200</code></pre>
Similarly for (user, client):
<pre class="line-numbers"><code class="language-bash"> > bin/kafka-configs.sh --bootstrap-server localhost:9092 --describe --entity-type users --entity-type clients
Configs for user-principal 'user1', default client-id are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200
Configs for user-principal 'user1', client-id 'clientA' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200</code></pre>
<h3 class="anchor-heading"><a id="datacenters" class="anchor-link"></a><a href="#datacenters">6.2 Datacenters</a></h3>
Some deployments will need to manage a data pipeline that spans multiple datacenters. Our recommended approach to this is to deploy a local Kafka cluster in each datacenter, with application instances in each datacenter interacting only with their local cluster and mirroring data between clusters (see the documentation on <a href="#georeplication">Geo-Replication</a> for how to do this).
<p>
This deployment pattern allows datacenters to act as independent entities and allows us to manage and tune inter-datacenter replication centrally. This allows each facility to stand alone and operate even if the inter-datacenter links are unavailable: when this occurs the mirroring falls behind until the link is restored at which time it catches up.
<p>
For applications that need a global view of all data you can use mirroring to provide clusters which have aggregate data mirrored from the local clusters in <i>all</i> datacenters. These aggregate clusters are used for reads by applications that require the full data set.
<p>
This is not the only possible deployment pattern. It is possible to read from or write to a remote Kafka cluster over the WAN, though obviously this will add whatever latency is required to get the cluster.
<p>
Kafka naturally batches data in both the producer and consumer so it can achieve high-throughput even over a high-latency connection. To allow this though it may be necessary to increase the TCP socket buffer sizes for the producer, consumer, and broker using the <code>socket.send.buffer.bytes</code> and <code>socket.receive.buffer.bytes</code> configurations. The appropriate way to set this is documented <a href="http://en.wikipedia.org/wiki/Bandwidth-delay_product">here</a>.
<p>
It is generally <i>not</i> advisable to run a <i>single</i> Kafka cluster that spans multiple datacenters over a high-latency link. This will incur very high replication latency both for Kafka writes and ZooKeeper writes, and neither Kafka nor ZooKeeper will remain available in all locations if the network between locations is unavailable.
<h3 class="anchor-heading"><a id="georeplication" class="anchor-link"></a><a href="#georeplication">6.3 Geo-Replication (Cross-Cluster Data Mirroring)</a></h3>
<h4 class="anchor-heading"><a id="georeplication-overview" class="anchor-link"></a><a href="#georeplication-overview">Geo-Replication Overview</a></h4>
<p>
Kafka administrators can define data flows that cross the boundaries of individual Kafka clusters, data centers, or geo-regions. Such event streaming setups are often needed for organizational, technical, or legal requirements. Common scenarios include:
</p>
<ul>
<li>Geo-replication</li>
<li>Disaster recovery</li>
<li>Feeding edge clusters into a central, aggregate cluster</li>
<li>Physical isolation of clusters (such as production vs. testing)</li>
<li>Cloud migration or hybrid cloud deployments</li>
<li>Legal and compliance requirements</li>
</ul>
<p>
Administrators can set up such inter-cluster data flows with Kafka's MirrorMaker (version 2), a tool to replicate data between different Kafka environments in a streaming manner. MirrorMaker is built on top of the Kafka Connect framework and supports features such as:
</p>
<ul>
<li>Replicates topics (data plus configurations)</li>
<li>Replicates consumer groups including offsets to migrate applications between clusters</li>
<li>Replicates ACLs</li>
<li>Preserves partitioning</li>
<li>Automatically detects new topics and partitions</li>
<li>Provides a wide range of metrics, such as end-to-end replication latency across multiple data centers/clusters</li>
<li>Fault-tolerant and horizontally scalable operations</li>
</ul>
<p>
<em>Note: Geo-replication with MirrorMaker replicates data across Kafka clusters. This inter-cluster replication is different from Kafka's <a href="#replication">intra-cluster replication</a>, which replicates data within the same Kafka cluster.</em>
</p>
<h4 class="anchor-heading"><a id="georeplication-flows" class="anchor-link"></a><a href="#georeplication-flows">What Are Replication Flows</a></h4>
<p>
With MirrorMaker, Kafka administrators can replicate topics, topic configurations, consumer groups and their offsets, and ACLs from one or more source Kafka clusters to one or more target Kafka clusters, i.e., across cluster environments. In a nutshell, MirrorMaker uses Connectors to consume from source clusters and produce to target clusters.
</p>
<p>
These directional flows from source to target clusters are called replication flows. They are defined with the format <code>{source_cluster}->{target_cluster}</code> in the MirrorMaker configuration file as described later. Administrators can create complex replication topologies based on these flows.
</p>
<p>
Here are some example patterns:
</p>
<ul>
<li>Active/Active high availability deployments: <code>A->B, B->A</code></li>
<li>Active/Passive or Active/Standby high availability deployments: <code>A->B</code></li>
<li>Aggregation (e.g., from many clusters to one): <code>A->K, B->K, C->K</code></li>
<li>Fan-out (e.g., from one to many clusters): <code>K->A, K->B, K->C</code></li>
<li>Forwarding: <code>A->B, B->C, C->D</code></li>
</ul>
<p>
By default, a flow replicates all topics and consumer groups (except excluded ones). 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.enabled = 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.enabled = 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-exactly_once" class="anchor-link"></a><a href="#georeplication-exactly_once">Exactly once</a></h5>
<p>
Exactly-once semantics are supported for dedicated MirrorMaker clusters as of version 3.5.0.</p>
<p>
For new MirrorMaker clusters, set the <code>exactly.once.source.support</code> property to enabled for all targeted Kafka clusters that should be written to with exactly-once semantics. For example, to enable exactly-once for writes to cluster </code>us-east</code>, the following configuration can be used:
</p>
<pre class="line-numbers"><code class="language-text">us-east.exactly.once.source.support = enabled
</code></pre>
<p>
For existing MirrorMaker clusters, a two-step upgrade is necessary. Instead of immediately setting the <code>exactly.once.source.support</code> property to enabled, first set it to <code>preparing</code> on all nodes in the cluster. Once this is complete, it can be set to </code>enabled</code> on all nodes in the cluster, in a second round of restarts.
</p>
<p>
In either case, it is also necessary to enable intra-cluster communication between the MirrorMaker nodes, as described in <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-710%3A+Full+support+for+distributed+mode+in+dedicated+MirrorMaker+2.0+clusters">KIP-710</a>. To do this, the <code>dedicated.mode.enable.internal.rest</code> property must be set to <code>true</code>. In addition, many of the REST-related <a href="https://kafka.apache.org/documentation/#connectconfigs">configuration properties available for Kafka Connect</a> can be specified the MirrorMaker config. For example, to enable intra-cluster communication in MirrorMaker cluster with each node listening on port 8080 of their local machine, the following should be added to the MirrorMaker config file:
</p>
<pre class="line-numbers"><code class="language-text">dedicated.mode.enable.internal.rest = true
listeners = http://localhost:8080
</code></pre>
<p><b>
Note that, if intra-cluster communication is enabled in production environments, it is highly recommended to secure the REST servers brought up by each MirrorMaker node. See the <a href="https://kafka.apache.org/documentation/#connectconfigs">configuration properties for Kafka Connect</a> for information on how this can be accomplished.
</b></p>
<p>
It is also recommended to filter records from aborted transactions out from replicated data when running MirrorMaker. To do this, ensure that the consumer used to read from source clusters is configured with <code>isolation.level</code> set to <code>read_committed</code>. If replicating data from cluster <code>us-west</code>, this can be done for all replication flows that read from that cluster by adding the following to the MirrorMaker config file:
</p>
<pre class="line-numbers"><code class="language-text">us-west.consumer.isolation.level = read_committed
</code></pre>
<p>
As a final note, under the hood, MirrorMaker uses Kafka Connect source connectors to replicate data. For more information on exactly-once support for these kinds of connectors, see the <a href="https://kafka.apache.org/documentation/#connect_exactlyoncesource">relevant docs page</a>.
</p>
<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.enabled = true
</code></pre>
<p>
By default, a replication flow will replicate all but a few special topics and consumer groups from the source cluster to the target cluster, and automatically detect any newly created topics and groups. The names of replicated topics in the target cluster will be prefixed with the name of the source cluster (see section further below). For example, the topic <code>foo</code> in the source cluster <code>us-west</code> would be replicated to a topic named <code>us-west.foo</code> in the target cluster <code>us-east</code>.
</p>
<p>
The subsequent sections explain how to customize this basic setup according to your needs.
</p>
<h5 class="anchor-heading"><a id="georeplication-flow-configure" class="anchor-link"></a><a href="#georeplication-flow-configure">Configuring Replication Flows</a></h5>
<p>
The configuration of a replication flow is a combination of top-level default settings (e.g., <code>topics</code>), on top of which flow-specific settings, if any, are applied (e.g., <code>us-west->us-east.topics</code>). To change the top-level defaults, add the respective top-level setting to the MirrorMaker configuration file. To override the defaults for a specific replication flow only, use the syntax format <code>{source}->{target}.{config.name}</code>.
</p>
<p>
The most important settings are:
</p>
<ul>
<li><code>topics</code>: list of topics or a regular expression that defines which topics in the source cluster to replicate (default: <code>topics = .*</code>)
<li><code>topics.exclude</code>: list of topics or a regular expression to subsequently exclude topics that were matched by the <code>topics</code> setting (default: <code>topics.exclude = .*[\-\.]internal, .*\.replica, __.*</code>)
<li><code>groups</code>: list of topics or regular expression that defines which consumer groups in the source cluster to replicate (default: <code>groups = .*</code>)
<li><code>groups.exclude</code>: list of topics or a regular expression to subsequently exclude consumer groups that were matched by the <code>groups</code> setting (default: <code>groups.exclude = console-consumer-.*, connect-.*, __.*</code>)
<li><code>{source}->{target}.enable</code>: set to <code>true</code> to enable the replication flow (default: <code>false</code>)
</ul>
<p>
Example:
</p>
<pre class="line-numbers"><code class="language-text"># Custom top-level defaults that apply to all replication flows
topics = .*
groups = consumer-group1, consumer-group2
# Don't forget to enable a flow!
us-west->us-east.enabled = 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 which can be left with their default values in most cases. See <a href="/documentation/#mirrormakerconfigs">MirrorMaker Configs</a>.
</p>
<h5 class="anchor-heading"><a id="georeplication-flow-secure" class="anchor-link"></a><a href="#georeplication-flow-secure">Securing Replication Flows</a></h5>
<p>
MirrorMaker supports the same <a href="#connectconfigs">security settings as Kafka Connect</a>, so please refer to the linked section for further information.
</p>
<p>
Example: Encrypt communication between MirrorMaker and the <code>us-east</code> cluster.
</p>
<pre class="line-numbers"><code class="language-text">us-east.security.protocol=SSL
us-east.ssl.truststore.location=/path/to/truststore.jks
us-east.ssl.truststore.password=my-secret-password
us-east.ssl.keystore.location=/path/to/keystore.jks
us-east.ssl.keystore.password=my-secret-password
us-east.ssl.key.password=my-secret-password
</code></pre>
<h5 class="anchor-heading"><a id="georeplication-topic-naming" class="anchor-link"></a><a href="#georeplication-topic-naming">Custom Naming of Replicated Topics in Target Clusters</a></h5>
<p>
Replicated topics in a target cluster—sometimes called <em>remote</em> topics—are renamed according to a replication policy. MirrorMaker uses this policy to ensure that events (aka records, messages) from different clusters are not written to the same topic-partition. By default as per <a href="https://github.com/apache/kafka/blob/trunk/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/DefaultReplicationPolicy.java">DefaultReplicationPolicy</a>, the names of replicated topics in the target clusters have the format <code>{source}.{source_topic_name}</code>:
</p>
<pre class="line-numbers"><code class="language-text">us-west us-east
========= =================
bar-topic
foo-topic --> us-west.foo-topic
</code></pre>
<p>
You can customize the separator (default: <code>.</code>) with the <code>replication.policy.separator</code> setting:
</p>
<pre class="line-numbers"><code class="language-text"># Defining a custom separator
us-west->us-east.replication.policy.separator = _
</code></pre>
<p>
If you need further control over how replicated topics are named, you can implement a custom <code>ReplicationPolicy</code> and override <code>replication.policy.class</code> (default is <code>DefaultReplicationPolicy</code>) in the MirrorMaker configuration.
</p>
<h5 class="anchor-heading"><a id="georeplication-config-conflicts" class="anchor-link"></a><a href="#georeplication-config-conflicts">Preventing Configuration Conflicts</a></h5>
<p>
MirrorMaker processes share configuration via their target Kafka clusters. This behavior may cause conflicts when configurations differ among MirrorMaker processes that operate against the same target cluster.
</p>
<p>
For example, the following two MirrorMaker processes would be racy:
</p>
<pre class="line-numbers"><code class="language-text"># Configuration of process 1
A->B.enabled = true
A->B.topics = foo
# Configuration of process 2
A->B.enabled = true
A->B.topics = bar
</code></pre>
<p>
In this case, the two processes will share configuration via cluster <code>B</code>, which causes a conflict. Depending on which of the two processes is the elected "leader", the result will be that either the topic <code>foo</code> or the topic <code>bar</code> is replicated, but not both.
</p>
<p>
It is therefore important to keep the MirrorMaker configuration 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>