forked from apache/kafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathops.html
1546 lines (1408 loc) · 95.2 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.
-->
<script id="ops-template" type="text/x-handlebars-template">
Here is some information on actually running Kafka as a production system based on usage and experience at LinkedIn. Please send us any additional tips you know of.
<h3><a id="basic_ops" 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><a id="basic_ops_add_topic" 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="brush: bash;">
> bin/kafka-topics.sh --zookeeper zk_host:port/chroot --create --topic my_topic_name
--partitions 20 --replication-factor 3 --config x=y
</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><a id="basic_ops_modify_topic" 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="brush: bash;">
> bin/kafka-topics.sh --zookeeper zk_host:port/chroot --alter --topic my_topic_name
--partitions 40
</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="brush: bash;">
> bin/kafka-configs.sh --zookeeper zk_host:port/chroot --entity-type topics --entity-name my_topic_name --alter --add-config x=y
</pre>
To remove a config:
<pre class="brush: bash;">
> bin/kafka-configs.sh --zookeeper zk_host:port/chroot --entity-type topics --entity-name my_topic_name --alter --delete-config x
</pre>
And finally deleting a topic:
<pre class="brush: bash;">
> bin/kafka-topics.sh --zookeeper zk_host:port/chroot --delete --topic my_topic_name
</pre>
Topic deletion option is disabled by default. To enable it set the server config
<pre class="brush: text;">delete.topic.enable=true</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><a id="basic_ops_restarting" 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="brush: text;">
controlled.shutdown.enable=true
</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><a id="basic_ops_leader_balancing" 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. This means that by default 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. You can have the Kafka cluster try to restore leadership to the restored replicas by running the command:
<pre class="brush: bash;">
> bin/kafka-preferred-replica-election.sh --zookeeper zk_host:port/chroot
</pre>
Since running this command can be tedious you can also configure Kafka to do this automatically by setting the following configuration:
<pre class="brush: text;">
auto.leader.rebalance.enable=true
</pre>
<h4><a id="basic_ops_racks" 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="brush: text;"> broker.rack=my-rack-id</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><a id="basic_ops_mirror_maker" href="#basic_ops_mirror_maker">Mirroring data between clusters</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="brush: bash;">
> bin/kafka-mirror-maker.sh
--consumer.config consumer.properties
--producer.config producer.properties --whitelist my-topic
</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 to expand it as a file path. For convenience we allow the use of ',' instead of '|' to specify a list of topics.
<p>
Sometimes it is easier to say what it is that you <i>don't</i> want. Instead of using <code>--whitelist</code> to say what you want
to mirror you can use <code>--blacklist</code> to say what to exclude. This also takes a regular expression argument.
However, <code>--blacklist</code> is not supported when the new consumer has been enabled (i.e. when <code>bootstrap.servers</code>
has been defined in the consumer configuration).
<p>
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.
<h4><a id="basic_ops_consumer_lag" 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="brush: bash;">
> bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zookeeper localhost:2181 --group test
Group Topic Pid Offset logSize Lag Owner
my-group my-topic 0 0 0 0 test_jkreps-mn-1394154511599-60744496-0
my-group my-topic 1 0 0 0 test_jkreps-mn-1394154521217-1a0be913-0
</pre>
NOTE: Since 0.9.0.0, the kafka.tools.ConsumerOffsetChecker tool has been deprecated. You should use the kafka.admin.ConsumerGroupCommand (or the bin/kafka-consumer-groups.sh script) to manage consumer groups, including consumers created with the <a href="http://kafka.apache.org/documentation.html#newconsumerapi">new consumer API</a>.
<h4><a id="basic_ops_consumer_group" href="#basic_ops_consumer_group">Managing Consumer Groups</a></h4>
With the ConsumerGroupCommand tool, we can list, describe, or delete consumer groups. Note that deletion is only available when the group metadata is stored in
ZooKeeper. When using the <a href="http://kafka.apache.org/documentation.html#newconsumerapi">new consumer API</a> (where
the broker handles coordination of partition handling and rebalance), the group is deleted when the last committed offset for that group expires.
For example, to list all consumer groups across all topics:
<pre class="brush: bash;">
> bin/kafka-consumer-groups.sh --bootstrap-server broker1:9092 --list
test-consumer-group
</pre>
To view offsets as in the previous example with the ConsumerOffsetChecker, we "describe" the consumer group like this:
<pre class="brush: bash;">
> bin/kafka-consumer-groups.sh --bootstrap-server broker1:9092 --describe --group test-consumer-group
TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID HOST CLIENT-ID
test-foo 0 1 3 2 consumer-1-a5d61779-4d04-4c50-a6d6-fb35d942642d /127.0.0.1 consumer-1
</pre>
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="brush: bash;">
> bin/kafka-consumer-groups.sh --zookeeper localhost:2181 --list
</pre>
<h4><a id="basic_ops_cluster_expansion" 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><a id="basic_ops_automigrate" 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="brush: bash;">
> cat topics-to-move.json
{"topics": [{"topic": "foo1"},
{"topic": "foo2"}],
"version":1
}
</pre>
Once the json file is ready, use the partition reassignment tool to generate a candidate assignment:
<pre class="brush: bash;">
> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --topics-to-move-json-file topics-to-move.json --broker-list "5,6" --generate
Current partition replica assignment
{"version":1,
"partitions":[{"topic":"foo1","partition":2,"replicas":[1,2]},
{"topic":"foo1","partition":0,"replicas":[3,4]},
{"topic":"foo2","partition":2,"replicas":[1,2]},
{"topic":"foo2","partition":0,"replicas":[3,4]},
{"topic":"foo1","partition":1,"replicas":[2,3]},
{"topic":"foo2","partition":1,"replicas":[2,3]}]
}
Proposed partition reassignment configuration
{"version":1,
"partitions":[{"topic":"foo1","partition":2,"replicas":[5,6]},
{"topic":"foo1","partition":0,"replicas":[5,6]},
{"topic":"foo2","partition":2,"replicas":[5,6]},
{"topic":"foo2","partition":0,"replicas":[5,6]},
{"topic":"foo1","partition":1,"replicas":[5,6]},
{"topic":"foo2","partition":1,"replicas":[5,6]}]
}
</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="brush: bash;">
> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file expand-cluster-reassignment.json --execute
Current partition replica assignment
{"version":1,
"partitions":[{"topic":"foo1","partition":2,"replicas":[1,2]},
{"topic":"foo1","partition":0,"replicas":[3,4]},
{"topic":"foo2","partition":2,"replicas":[1,2]},
{"topic":"foo2","partition":0,"replicas":[3,4]},
{"topic":"foo1","partition":1,"replicas":[2,3]},
{"topic":"foo2","partition":1,"replicas":[2,3]}]
}
Save this to use as the --reassignment-json-file option during rollback
Successfully started reassignment of partitions
{"version":1,
"partitions":[{"topic":"foo1","partition":2,"replicas":[5,6]},
{"topic":"foo1","partition":0,"replicas":[5,6]},
{"topic":"foo2","partition":2,"replicas":[5,6]},
{"topic":"foo2","partition":0,"replicas":[5,6]},
{"topic":"foo1","partition":1,"replicas":[5,6]},
{"topic":"foo2","partition":1,"replicas":[5,6]}]
}
</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="brush: bash;">
> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file expand-cluster-reassignment.json --verify
Status of partition reassignment:
Reassignment of partition [foo1,0] completed successfully
Reassignment of partition [foo1,1] is in progress
Reassignment of partition [foo1,2] is in progress
Reassignment of partition [foo2,0] completed successfully
Reassignment of partition [foo2,1] completed successfully
Reassignment of partition [foo2,2] completed successfully
</pre>
<h5><a id="basic_ops_partitionassignment" 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="brush: bash;">
> cat custom-reassignment.json
{"version":1,"partitions":[{"topic":"foo1","partition":0,"replicas":[5,6]},{"topic":"foo2","partition":1,"replicas":[2,3]}]}
</pre>
Then, use the json file with the --execute option to start the reassignment process:
<pre class="brush: bash;">
> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file custom-reassignment.json --execute
Current partition replica assignment
{"version":1,
"partitions":[{"topic":"foo1","partition":0,"replicas":[1,2]},
{"topic":"foo2","partition":1,"replicas":[3,4]}]
}
Save this to use as the --reassignment-json-file option during rollback
Successfully started reassignment of partitions
{"version":1,
"partitions":[{"topic":"foo1","partition":0,"replicas":[5,6]},
{"topic":"foo2","partition":1,"replicas":[2,3]}]
}
</pre>
<p>
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="brush: bash;">
> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file custom-reassignment.json --verify
Status of partition reassignment:
Reassignment of partition [foo1,0] completed successfully
Reassignment of partition [foo2,1] completed successfully
</pre>
<h4><a id="basic_ops_decommissioning_brokers" 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><a id="basic_ops_increase_replication_factor" 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="brush: bash;">
> cat increase-replication-factor.json
{"version":1,
"partitions":[{"topic":"foo","partition":0,"replicas":[5,6,7]}]}
</pre>
Then, use the json file with the --execute option to start the reassignment process:
<pre class="brush: bash;">
> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file increase-replication-factor.json --execute
Current partition replica assignment
{"version":1,
"partitions":[{"topic":"foo","partition":0,"replicas":[5]}]}
Save this to use as the --reassignment-json-file option during rollback
Successfully started reassignment of partitions
{"version":1,
"partitions":[{"topic":"foo","partition":0,"replicas":[5,6,7]}]}
</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="brush: bash;">
> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file increase-replication-factor.json --verify
Status of partition reassignment:
Reassignment of partition [foo,0] completed successfully
</pre>
You can also verify the increase in replication factor with the kafka-topics tool:
<pre class="brush: bash;">
> bin/kafka-topics.sh --zookeeper localhost:2181 --topic foo --describe
Topic:foo PartitionCount:1 ReplicationFactor:3 Configs:
Topic: foo Partition: 0 Leader: 5 Replicas: 5,6,7 Isr: 5,6,7
</pre>
<h4><a id="rep-throttle" 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="brush: bash;">$ bin/kafka-reassign-partitions.sh --zookeeper myhost:2181--execute --reassignment-json-file bigger-cluster.json —throttle 50000000</pre>
When you execute this script you will see the throttle engage:
<pre class="brush: bash;">
The throttle limit was set to 50000000 B/s
Successfully started reassignment of partitions.</pre>
<p>Should you wish to alter the throttle, during a rebalance, say to increase the throughput so it completes quicker, you can do this by re-running the execute command passing the same reassignment-json-file:</p>
<pre class="brush: bash;">$ bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --execute --reassignment-json-file bigger-cluster.json --throttle 700000000
There is an existing assignment running.
The throttle limit was set to 700000000 B/s</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="brush: bash;">
> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --verify --reassignment-json-file bigger-cluster.json
Status of partition reassignment:
Reassignment of partition [my-topic,1] completed successfully
Reassignment of partition [mytopic,0] completed successfully
Throttle was removed.</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. The throttle value itself. This is configured, at a broker
level, using the dynamic properties: </p>
<pre class="brush: text;">leader.replication.throttled.rate
follower.replication.throttled.rate</pre>
<p>There is also an enumerated set of throttled replicas: </p>
<pre class="brush: text;">leader.replication.throttled.replicas
follower.replication.throttled.replicas</pre>
<p>Which are configured per topic. 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="brush: bash;">
> bin/kafka-configs.sh --describe --zookeeper localhost:2181 --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</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="brush: bash;">
> bin/kafka-configs.sh --describe --zookeeper localhost:2181 --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</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
—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</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]+)</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><a id="quotas" 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="brush: bash;">
> bin/kafka-configs.sh --zookeeper localhost:2181 --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'.
</pre>
Configure custom quota for user=user1:
<pre class="brush: bash;">
> bin/kafka-configs.sh --zookeeper localhost:2181 --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'.
</pre>
Configure custom quota for client-id=clientA:
<pre class="brush: bash;">
> bin/kafka-configs.sh --zookeeper localhost:2181 --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'.
</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="brush: bash;">
> bin/kafka-configs.sh --zookeeper localhost:2181 --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.
</pre>
Configure default quota for user:
<pre class="brush: bash;">
> bin/kafka-configs.sh --zookeeper localhost:2181 --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.
</pre>
Configure default quota for client-id:
<pre class="brush: bash;">
> bin/kafka-configs.sh --zookeeper localhost:2181 --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.
</pre>
Here's how to describe the quota for a given (user, client-id):
<pre class="brush: bash;">
> bin/kafka-configs.sh --zookeeper localhost:2181 --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
</pre>
Describe quota for a given user:
<pre class="brush: bash;">
> bin/kafka-configs.sh --zookeeper localhost:2181 --describe --entity-type users --entity-name user1
Configs for user-principal 'user1' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200
</pre>
Describe quota for a given client-id:
<pre class="brush: bash;">
> bin/kafka-configs.sh --zookeeper localhost:2181 --describe --entity-type clients --entity-name clientA
Configs for client-id 'clientA' are producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200
</pre>
If entity name is not specified, all entities of the specified type are described. For example, describe all users:
<pre class="brush: bash;">
> bin/kafka-configs.sh --zookeeper localhost:2181 --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
</pre>
Similarly for (user, client):
<pre class="brush: bash;">
> bin/kafka-configs.sh --zookeeper localhost:2181 --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
</pre>
<p>
It is possible to set default quotas that apply to all client-ids by setting these configs on the brokers. These properties are applied only if quota overrides or defaults are not configured in Zookeeper. By default, each client-id receives an unlimited quota. The following sets the default quota per producer and consumer client-id to 10MB/sec.
<pre class="brush: text;">
quota.producer.default=10485760
quota.consumer.default=10485760
</pre>
Note that these properties are being deprecated and may be removed in a future release. Defaults configured using kafka-configs.sh take precedence over these properties.
<h3><a id="datacenters" 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).
<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><a id="config" href="#config">6.3 Kafka Configuration</a></h3>
<h4><a id="clientconfig" href="#clientconfig">Important Client Configurations</a></h4>
The most important old Scala producer configurations control
<ul>
<li>acks</li>
<li>compression</li>
<li>sync vs async production</li>
<li>batch size (for async producers)</li>
</ul>
The most important new Java producer configurations control
<ul>
<li>acks</li>
<li>compression</li>
<li>batch size</li>
</ul>
The most important consumer configuration is the fetch size.
<p>
All configurations are documented in the <a href="#configuration">configuration</a> section.
<p>
<h4><a id="prodconfig" href="#prodconfig">A Production Server Config</a></h4>
Here is an example production server configuration:
<pre class="brush: text;">
# ZooKeeper
zookeeper.connect=[list of ZooKeeper servers]
# Log configuration
num.partitions=8
default.replication.factor=3
log.dir=[List of directories. Kafka should have its own dedicated disk(s) or SSD(s).]
# Other configurations
broker.id=[An integer. Start with 0 and increment by 1 for each new broker.]
listeners=[list of listeners]
auto.create.topics.enable=false
min.insync.replicas=2
queued.max.requests=[number of concurrent requests]
</pre>
Our client configuration varies a fair amount between different use cases.
<h3><a id="java" href="#java">6.4 Java Version</a></h3>
From a security perspective, we recommend you use the latest released version of JDK 1.8 as older freely available versions have disclosed security vulnerabilities.
LinkedIn is currently running JDK 1.8 u5 (looking to upgrade to a newer version) with the G1 collector. If you decide to use the G1 collector (the current default) and you are still on JDK 1.7, make sure you are on u51 or newer. LinkedIn tried out u21 in testing, but they had a number of problems with the GC implementation in that version.
LinkedIn's tuning looks like this:
<pre class="brush: text;">
-Xmx6g -Xms6g -XX:MetaspaceSize=96m -XX:+UseG1GC
-XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:G1HeapRegionSize=16M
-XX:MinMetaspaceFreeRatio=50 -XX:MaxMetaspaceFreeRatio=80
</pre>
For reference, here are the stats on one of LinkedIn's busiest clusters (at peak):
<ul>
<li>60 brokers</li>
<li>50k partitions (replication factor 2)</li>
<li>800k messages/sec in</li>
<li>300 MB/sec inbound, 1 GB/sec+ outbound</li>
</ul>
The tuning looks fairly aggressive, but all of the brokers in that cluster have a 90% GC pause time of about 21ms, and they're doing less than 1 young GC per second.
<h3><a id="hwandos" href="#hwandos">6.5 Hardware and OS</a></h3>
We are using dual quad-core Intel Xeon machines with 24GB of memory.
<p>
You need sufficient memory to buffer active readers and writers. You can do a back-of-the-envelope estimate of memory needs by assuming you want to be able to buffer for 30 seconds and compute your memory need as write_throughput*30.
<p>
The disk throughput is important. We have 8x7200 rpm SATA drives. In general disk throughput is the performance bottleneck, and more disks is better. Depending on how you configure flush behavior you may or may not benefit from more expensive disks (if you force flush often then higher RPM SAS drives may be better).
<h4><a id="os" href="#os">OS</a></h4>
Kafka should run well on any unix system and has been tested on Linux and Solaris.
<p>
We have seen a few issues running on Windows and Windows is not currently a well supported platform though we would be happy to change that.
<p>
It is unlikely to require much OS-level tuning, but there are two potentially important OS-level configurations:
<ul>
<li>File descriptor limits: Kafka uses file descriptors for log segments and open connections. If a broker hosts many partitions, consider that the broker needs at least (number_of_partitions)*(partition_size/segment_size) to track all log segments in addition to the number of connections the broker makes. We recommend at least 100000 allowed file descriptors for the broker processes as a starting point.
<li>Max socket buffer size: can be increased to enable high-performance data transfer between data centers as <a href="http://www.psc.edu/index.php/networking/641-tcp-tune">described here</a>.
</ul>
<p>
<h4><a id="diskandfs" href="#diskandfs">Disks and Filesystem</a></h4>
We recommend using multiple drives to get good throughput and not sharing the same drives used for Kafka data with application logs or other OS filesystem activity to ensure good latency. You can either RAID these drives together into a single volume or format and mount each drive as its own directory. Since Kafka has replication the redundancy provided by RAID can also be provided at the application level. This choice has several tradeoffs.
<p>
If you configure multiple data directories partitions will be assigned round-robin to data directories. Each partition will be entirely in one of the data directories. If data is not well balanced among partitions this can lead to load imbalance between disks.
<p>
RAID can potentially do better at balancing load between disks (although it doesn't always seem to) because it balances load at a lower level. The primary downside of RAID is that it is usually a big performance hit for write throughput and reduces the available disk space.
<p>
Another potential benefit of RAID is the ability to tolerate disk failures. However our experience has been that rebuilding the RAID array is so I/O intensive that it effectively disables the server, so this does not provide much real availability improvement.
<h4><a id="appvsosflush" href="#appvsosflush">Application vs. OS Flush Management</a></h4>
Kafka always immediately writes all data to the filesystem and supports the ability to configure the flush policy that controls when data is forced out of the OS cache and onto disk using the flush. This flush policy can be controlled to force data to disk after a period of time or after a certain number of messages has been written. There are several choices in this configuration.
<p>
Kafka must eventually call fsync to know that data was flushed. When recovering from a crash for any log segment not known to be fsync'd Kafka will check the integrity of each message by checking its CRC and also rebuild the accompanying offset index file as part of the recovery process executed on startup.
<p>
Note that durability in Kafka does not require syncing data to disk, as a failed node will always recover from its replicas.
<p>
We recommend using the default flush settings which disable application fsync entirely. This means relying on the background flush done by the OS and Kafka's own background flush. This provides the best of all worlds for most uses: no knobs to tune, great throughput and latency, and full recovery guarantees. We generally feel that the guarantees provided by replication are stronger than sync to local disk, however the paranoid still may prefer having both and application level fsync policies are still supported.
<p>
The drawback of using application level flush settings is that it is less efficient in its disk usage pattern (it gives the OS less leeway to re-order writes) and it can introduce latency as fsync in most Linux filesystems blocks writes to the file whereas the background flushing does much more granular page-level locking.
<p>
In general you don't need to do any low-level tuning of the filesystem, but in the next few sections we will go over some of this in case it is useful.
<h4><a id="linuxflush" href="#linuxflush">Understanding Linux OS Flush Behavior</a></h4>
In Linux, data written to the filesystem is maintained in <a href="http://en.wikipedia.org/wiki/Page_cache">pagecache</a> until it must be written out to disk (due to an application-level fsync or the OS's own flush policy). The flushing of data is done by a set of background threads called pdflush (or in post 2.6.32 kernels "flusher threads").
<p>
Pdflush has a configurable policy that controls how much dirty data can be maintained in cache and for how long before it must be written back to disk.
This policy is described <a href="http://web.archive.org/web/20160518040713/http://www.westnet.com/~gsmith/content/linux-pdflush.htm">here</a>.
When Pdflush cannot keep up with the rate of data being written it will eventually cause the writing process to block incurring latency in the writes to slow down the accumulation of data.
<p>
You can see the current state of OS memory usage by doing
<pre class="brush: bash;"> > cat /proc/meminfo </pre>
The meaning of these values are described in the link above.
<p>
Using pagecache has several advantages over an in-process cache for storing data that will be written out to disk:
<ul>
<li>The I/O scheduler will batch together consecutive small writes into bigger physical writes which improves throughput.
<li>The I/O scheduler will attempt to re-sequence writes to minimize movement of the disk head which improves throughput.
<li>It automatically uses all the free memory on the machine
</ul>
<h4><a id="filesystems" href="#filesystems">Filesystem Selection</a></h4>
<p>Kafka uses regular files on disk, and as such it has no hard dependency on a specific filesystem. The two filesystems which have the most usage, however, are EXT4 and XFS. Historically, EXT4 has had more usage, but recent improvements to the XFS filesystem have shown it to have better performance characteristics for Kafka's workload with no compromise in stability.</p>
<p>Comparison testing was performed on a cluster with significant message loads, using a variety of filesystem creation and mount options. The primary metric in Kafka that was monitored was the "Request Local Time", indicating the amount of time append operations were taking. XFS resulted in much better local times (160ms vs. 250ms+ for the best EXT4 configuration), as well as lower average wait times. The XFS performance also showed less variability in disk performance.</p>
<h5><a id="generalfs" href="#generalfs">General Filesystem Notes</a></h5>
For any filesystem used for data directories, on Linux systems, the following options are recommended to be used at mount time:
<ul>
<li>noatime: This option disables updating of a file's atime (last access time) attribute when the file is read. This can eliminate a significant number of filesystem writes, especially in the case of bootstrapping consumers. Kafka does not rely on the atime attributes at all, so it is safe to disable this.</li>
</ul>
<h5><a id="xfs" href="#xfs">XFS Notes</a></h5>
The XFS filesystem has a significant amount of auto-tuning in place, so it does not require any change in the default settings, either at filesystem creation time or at mount. The only tuning parameters worth considering are:
<ul>
<li>largeio: This affects the preferred I/O size reported by the stat call. While this can allow for higher performance on larger disk writes, in practice it had minimal or no effect on performance.</li>
<li>nobarrier: For underlying devices that have battery-backed cache, this option can provide a little more performance by disabling periodic write flushes. However, if the underlying device is well-behaved, it will report to the filesystem that it does not require flushes, and this option will have no effect.</li>
</ul>
<h5><a id="ext4" href="#ext4">EXT4 Notes</a></h5>
EXT4 is a serviceable choice of filesystem for the Kafka data directories, however getting the most performance out of it will require adjusting several mount options. In addition, these options are generally unsafe in a failure scenario, and will result in much more data loss and corruption. For a single broker failure, this is not much of a concern as the disk can be wiped and the replicas rebuilt from the cluster. In a multiple-failure scenario, such as a power outage, this can mean underlying filesystem (and therefore data) corruption that is not easily recoverable. The following options can be adjusted:
<ul>
<li>data=writeback: Ext4 defaults to data=ordered which puts a strong order on some writes. Kafka does not require this ordering as it does very paranoid data recovery on all unflushed log. This setting removes the ordering constraint and seems to significantly reduce latency.
<li>Disabling journaling: Journaling is a tradeoff: it makes reboots faster after server crashes but it introduces a great deal of additional locking which adds variance to write performance. Those who don't care about reboot time and want to reduce a major source of write latency spikes can turn off journaling entirely.
<li>commit=num_secs: This tunes the frequency with which ext4 commits to its metadata journal. Setting this to a lower value reduces the loss of unflushed data during a crash. Setting this to a higher value will improve throughput.
<li>nobh: This setting controls additional ordering guarantees when using data=writeback mode. This should be safe with Kafka as we do not depend on write ordering and improves throughput and latency.
<li>delalloc: Delayed allocation means that the filesystem avoid allocating any blocks until the physical write occurs. This allows ext4 to allocate a large extent instead of smaller pages and helps ensure the data is written sequentially. This feature is great for throughput. It does seem to involve some locking in the filesystem which adds a bit of latency variance.
</ul>
<h3><a id="monitoring" href="#monitoring">6.6 Monitoring</a></h3>
Kafka uses Yammer Metrics for metrics reporting in the server and Scala clients. 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>
The easiest way to see the available metrics is to fire up jconsole and point it at a running kafka client or server; this will allow browsing all metrics with JMX.
<p>
We do graphing and alerting on the following metrics:
<table class="data-table">
<tbody><tr>
<th>Description</th>
<th>Mbean name</th>
<th>Normal value</th>
</tr>
<tr>
<td>Message in rate</td>
<td>kafka.server:type=BrokerTopicMetrics,name=MessagesInPerSec</td>
<td></td>
</tr>
<tr>
<td>Byte in rate</td>
<td>kafka.server:type=BrokerTopicMetrics,name=BytesInPerSec</td>
<td></td>
</tr>
<tr>
<td>Request rate</td>
<td>kafka.network:type=RequestMetrics,name=RequestsPerSec,request={Produce|FetchConsumer|FetchFollower}</td>
<td></td>
</tr>
<tr>
<td>Byte out rate</td>
<td>kafka.server:type=BrokerTopicMetrics,name=BytesOutPerSec</td>
<td></td>
</tr>
<tr>
<td>Log flush rate and time</td>
<td>kafka.log:type=LogFlushStats,name=LogFlushRateAndTimeMs</td>
<td></td>
</tr>
<tr>
<td># of under replicated partitions (|ISR| < |all replicas|)</td>
<td>kafka.server:type=ReplicaManager,name=UnderReplicatedPartitions</td>
<td>0</td>
</tr>
<tr>
<td># of under minIsr partitions (|ISR| < min.insync.replicas)</td>
<td>kafka.server:type=ReplicaManager,name=UnderMinIsrPartitionCount</td>
<td>0</td>
</tr>
<tr>
<td># of offline log directories</td>
<td>kafka.log:type=LogManager,name=OfflineLogDirectoryCount</td>
<td>0</td>
</tr>
<tr>
<td>Is controller active on broker</td>
<td>kafka.controller:type=KafkaController,name=ActiveControllerCount</td>
<td>only one broker in the cluster should have 1</td>
</tr>
<tr>
<td>Leader election rate</td>
<td>kafka.controller:type=ControllerStats,name=LeaderElectionRateAndTimeMs</td>
<td>non-zero when there are broker failures</td>
</tr>
<tr>
<td>Unclean leader election rate</td>
<td>kafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec</td>
<td>0</td>
</tr>
<tr>
<td>Partition counts</td>
<td>kafka.server:type=ReplicaManager,name=PartitionCount</td>
<td>mostly even across brokers</td>
</tr>
<tr>
<td>Leader replica counts</td>
<td>kafka.server:type=ReplicaManager,name=LeaderCount</td>
<td>mostly even across brokers</td>
</tr>
<tr>
<td>ISR shrink rate</td>
<td>kafka.server:type=ReplicaManager,name=IsrShrinksPerSec</td>
<td>If a broker goes down, ISR for some of the partitions will
shrink. When that broker is up again, ISR will be expanded
once the replicas are fully caught up. Other than that, the
expected value for both ISR shrink rate and expansion rate is 0. </td>
</tr>
<tr>
<td>ISR expansion rate</td>
<td>kafka.server:type=ReplicaManager,name=IsrExpandsPerSec</td>
<td>See above</td>
</tr>
<tr>
<td>Max lag in messages btw follower and leader replicas</td>
<td>kafka.server:type=ReplicaFetcherManager,name=MaxLag,clientId=Replica</td>
<td>lag should be proportional to the maximum batch size of a produce request.</td>
</tr>
<tr>
<td>Lag in messages per follower replica</td>
<td>kafka.server:type=FetcherLagMetrics,name=ConsumerLag,clientId=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)</td>
<td>lag should be proportional to the maximum batch size of a produce request.</td>
</tr>
<tr>
<td>Requests waiting in the producer purgatory</td>
<td>kafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation=Produce</td>
<td>non-zero if ack=-1 is used</td>
</tr>
<tr>
<td>Requests waiting in the fetch purgatory</td>
<td>kafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation=Fetch</td>
<td>size depends on fetch.wait.max.ms in the consumer</td>
</tr>
<tr>
<td>Request total time</td>
<td>kafka.network:type=RequestMetrics,name=TotalTimeMs,request={Produce|FetchConsumer|FetchFollower}</td>
<td>broken into queue, local, remote and response send time</td>
</tr>
<tr>
<td>Time the request waits in the request queue</td>
<td>kafka.network:type=RequestMetrics,name=RequestQueueTimeMs,request={Produce|FetchConsumer|FetchFollower}</td>
<td></td>
</tr>
<tr>
<td>Time the request is processed at the leader</td>
<td>kafka.network:type=RequestMetrics,name=LocalTimeMs,request={Produce|FetchConsumer|FetchFollower}</td>
<td></td>
</tr>
<tr>
<td>Time the request waits for the follower</td>
<td>kafka.network:type=RequestMetrics,name=RemoteTimeMs,request={Produce|FetchConsumer|FetchFollower}</td>
<td>non-zero for produce requests when ack=-1</td>
</tr>
<tr>
<td>Time the request waits in the response queue</td>
<td>kafka.network:type=RequestMetrics,name=ResponseQueueTimeMs,request={Produce|FetchConsumer|FetchFollower}</td>
<td></td>
</tr>
<tr>
<td>Time to send the response</td>
<td>kafka.network:type=RequestMetrics,name=ResponseSendTimeMs,request={Produce|FetchConsumer|FetchFollower}</td>
<td></td>
</tr>
<tr>
<td>Number of messages the consumer lags behind the producer by. Published by the consumer, not broker.</td>
<td>
<p><em>Old consumer:</em> kafka.consumer:type=ConsumerFetcherManager,name=MaxLag,clientId=([-.\w]+)</p>
<p><em>New consumer:</em> kafka.consumer:type=consumer-fetch-manager-metrics,client-id={client-id} Attribute: records-lag-max</p>
</td>
<td></td>
</tr>
<tr>
<td>The average fraction of time the network processors are idle</td>
<td>kafka.network:type=SocketServer,name=NetworkProcessorAvgIdlePercent</td>
<td>between 0 and 1, ideally > 0.3</td>
</tr>
<tr>
<td>The average fraction of time the request handler threads are idle</td>
<td>kafka.server:type=KafkaRequestHandlerPool,name=RequestHandlerAvgIdlePercent</td>
<td>between 0 and 1, ideally > 0.3</td>
</tr>
<tr>
<td>Bandwidth quota metrics per (user, client-id), user or client-id</td>
<td>kafka.server:type={Produce|Fetch},user=([-.\w]+),client-id=([-.\w]+)</td>
<td>Two attributes. throttle-time indicates the amount of time in ms the client was throttled. Ideally = 0.
byte-rate indicates the data produce/consume rate of the client in bytes/sec.
For (user, client-id) quotas, both user and client-id are specified. If per-client-id quota is applied to the client, user is not specified. If per-user quota is applied, client-id is not specified.</td>
</tr>
<tr>
<td>Request quota metrics per (user, client-id), user or client-id</td>
<td>kafka.server:type=Request,user=([-.\w]+),client-id=([-.\w]+)</td>
<td>Two attributes. throttle-time indicates the amount of time in ms the client was throttled. Ideally = 0.
request-time indicates the percentage of time spent in broker network and I/O threads to process requests from client group.
For (user, client-id) quotas, both user and client-id are specified. If per-client-id quota is applied to the client, user is not specified. If per-user quota is applied, client-id is not specified.</td>
</tr>
<tr>
<td>Requests exempt from throttling</td>
<td>kafka.server:type=Request</td>
<td>exempt-throttle-time indicates the percentage of time spent in broker network and I/O threads to process requests
that are exempt from throttling.</td>
</tr>
</tbody></table>
<h4><a id="selector_monitoring" href="#selector_monitoring">Common monitoring metrics for producer/consumer/connect/streams</a></h4>
The following metrics are available on producer/consumer/connector/streams instances. For specific metrics, please see following sections.
<table class="data-table">
<tbody>
<tr>
<th>Metric/Attribute name</th>
<th>Description</th>
<th>Mbean name</th>
</tr>
<tr>
<td>connection-close-rate</td>
<td>Connections closed per second in the window.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>connection-creation-rate</td>
<td>New connections established per second in the window.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>network-io-rate</td>
<td>The average number of network operations (reads or writes) on all connections per second.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>outgoing-byte-rate</td>
<td>The average number of outgoing bytes sent per second to all servers.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>request-rate</td>
<td>The average number of requests sent per second.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>request-size-avg</td>
<td>The average size of all requests in the window.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>request-size-max</td>
<td>The maximum size of any request sent in the window.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>incoming-byte-rate</td>
<td>Bytes/second read off all sockets.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>response-rate</td>
<td>Responses received sent per second.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>select-rate</td>
<td>Number of times the I/O layer checked for new I/O to perform per second.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>io-wait-time-ns-avg</td>
<td>The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>io-wait-ratio</td>
<td>The fraction of time the I/O thread spent waiting.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>io-time-ns-avg</td>
<td>The average length of time for I/O per select call in nanoseconds.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>io-ratio</td>
<td>The fraction of time the I/O thread spent doing I/O.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>connection-count</td>
<td>The current number of active connections.</td>
<td>kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)</td>
</tr>
</tbody>
</table>
<h4><a id="common_node_monitoring" href="#common_node_monitoring">Common Per-broker metrics for producer/consumer/connect/streams</a></h4>
The following metrics are available on producer/consumer/connector/streams instances. For specific metrics, please see following sections.
<table class="data-table">
<tbody>
<tr>
<th>Metric/Attribute name</th>
<th>Description</th>
<th>Mbean name</th>
</tr>
<tr>
<td>outgoing-byte-rate</td>
<td>The average number of outgoing bytes sent per second for a node.</td>
<td>kafka.producer:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>request-rate</td>
<td>The average number of requests sent per second for a node.</td>
<td>kafka.producer:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>request-size-avg</td>
<td>The average size of all requests in the window for a node.</td>
<td>kafka.producer:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>request-size-max</td>
<td>The maximum size of any request sent in the window for a node.</td>
<td>kafka.producer:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>incoming-byte-rate</td>
<td>The average number of responses received per second for a node.</td>
<td>kafka.producer:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>request-latency-avg</td>
<td>The average request latency in ms for a node.</td>
<td>kafka.producer:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)</td>
</tr>
<tr>
<td>request-latency-max</td>