如何实现kafka集群2个IDC高可靠部署

举报
Xiao_Chuan 发表于 2025/08/09 16:50:38 2025/08/09
【摘要】 验证kafka集群2AZ部署可行性

部署背景

目前有3个IDC,其中A城2个IDC相距<10km,时延<2ms;B城IDC与A城IDC时延>10ms。
A城业务需要kafka跨IDC部署满足高可靠,当1个IDC故障后,kafka集群仍然可以继续正常读写,RPO=0。

部署方案

因为B城IDC时延过大,不满足3个IDC组成分布式3AZ kafka集群部署条件。因此考虑A城两AZ部署,B城仅作为仲裁AZ。
这样zk仍然是标准的3AZ正常仲裁,核心业务数据也不会往B城的仲裁AZ同步,避免了网络时延导致的性能下降。

Kafka-2az.png

zookeeper核心配置

server.128=192.168.150.128:2000:3000
server.129=192.168.150.129:2000:3000
server.130=192.168.150.130:2000:3000

kafka核心配置

zookeeper.connect=192.168.150.129:2181,192.168.150.130:2181,192.168.150.128:2181

num.partitions=4

offsets.topic.replication.factor=4
transaction.state.log.replication.factor=4
transaction.state.log.min.isr=2

default.replications.factor=4
min.insync.replicas=2
replica.selector.class=org.apache.kafka.common.replica.RackAwareReplicaSelector
broker.rack=1或2

所有的*factor配置全部要求=4,和broker数量一致。然后配置min.insync.replicas=2,这样一旦1个IDC内的2个broker故障,存货IDC内还有2个broker可以满足副本读写同步要求。

配置broker.rack感知,让副本优先跨IDC,IDC1配置broker.rack=1,IDC2配置broker.rack=2。

部署测试

通过关闭部署kafka的虚拟机来模拟IDC故障。

正常集群

[root@localhost ~]# /opt/kafka_2.13-3.9.0/bin/kafka-broker-api-versions.sh --bootstrap-server 192.168.150.131:9092 |grep 9092
192.168.150.132:9092 (id: 4 rack: 2) -> (
192.168.150.131:9092 (id: 3 rack: 1) -> (
192.168.150.129:9092 (id: 1 rack: 1) -> (
192.168.150.130:9092 (id: 2 rack: 2) -> (

# 默认129(borker.id=1)是controller
[zk: 192.168.150.128:2181(CONNECTED) 6] get /controller
{"version":2,"brokerid":1,"timestamp":"1754720609029","kraftControllerEpoch":-1}
[zk: 192.168.150.128:2181(CONNECTED) 13] ls /brokers/ids
[1, 2, 3, 4]

# 针对topic1做读写测试
[zk: 192.168.150.128:2181(CONNECTED) 14] ls /brokers/topics
[__consumer_offsets, demo, topic1]
[zk: 192.168.150.128:2181(CONNECTED) 17] get /cluster/id
{"version":"1","id":"8Nfp4cHmRW2501p2n4L8wQ"}

# 4个borker都分配了副本Leader
[root@localhost ~]# /opt/kafka_2.13-3.9.0/bin/kafka-topics.sh --bootstrap-server 192.168.150.130:9092 --create --topic topic1 --replication-factor=4
Created topic topic1.
[root@localhost ~]# /opt/kafka_2.13-3.9.0/bin/kafka-topics.sh --bootstrap-server 192.168.150.130:9092 --describe --topic topic1
Topic: topic1	TopicId: P2nKvNOtTLe-ykL7mVcsJw	PartitionCount: 4	ReplicationFactor: 4	Configs: min.insync.replicas=2
	Topic: topic1	Partition: 0	Leader: 3	Replicas: 3,2,4,1	Isr: 3,2,4,1	Elr: N/A	LastKnownElr: N/A
	Topic: topic1	Partition: 1	Leader: 2	Replicas: 2,4,1,3	Isr: 2,4,1,3	Elr: N/A	LastKnownElr: N/A
	Topic: topic1	Partition: 2	Leader: 4	Replicas: 4,1,3,2	Isr: 4,1,3,2	Elr: N/A	LastKnownElr: N/A
	Topic: topic1	Partition: 3	Leader: 1	Replicas: 1,3,2,4	Isr: 1,3,2,4	Elr: N/A	LastKnownElr: N/A

模拟1个IDC故障

# 先故障129,可以看到Leader里找不到broker.id=1
# 故障129,第一个master
[root@localhost ~]# /opt/kafka_2.13-3.9.0/bin/kafka-topics.sh --bootstrap-server 192.168.150.130:9092 --describe --topic topic1
Topic: topic1 TopicId: P2nKvNOtTLe-ykL7mVcsJw PartitionCount: 4 ReplicationFactor: 4 Configs: min.insync.replicas=2
Topic: topic1 Partition: 0 Leader: 3 Replicas: 3,2,4,1 Isr: 3,2,4 Elr: N/A LastKnownElr: N/A
Topic: topic1 Partition: 1 Leader: 2 Replicas: 2,4,1,3 Isr: 2,4,3 Elr: N/A LastKnownElr: N/A
Topic: topic1 Partition: 2 Leader: 4 Replicas: 4,1,3,2 Isr: 4,3,2 Elr: N/A LastKnownElr: N/A
Topic: topic1 Partition: 3 Leader: 3 Replicas: 1,3,2,4 Isr: 3,2,4 Elr: N/A LastKnownElr: N/A
# controller切换到broker2
[zk: 192.168.150.128:2181(CONNECTED) 23] get /controller
{"version":2,"brokerid":2,"timestamp":"1754723996337","kraftControllerEpoch":-1}
# 故障131,可以看到Leader里找不到broker.id=3
[root@localhost ~]# /opt/kafka_2.13-3.9.0/bin/kafka-topics.sh --bootstrap-server 192.168.150.130:9092 --describe --topic topic1
Topic: topic1 TopicId: P2nKvNOtTLe-ykL7mVcsJw PartitionCount: 4 ReplicationFactor: 4 Configs: min.insync.replicas=2
Topic: topic1 Partition: 0 Leader: 2 Replicas: 3,2,4,1 Isr: 2,4 Elr: N/A LastKnownElr: N/A
Topic: topic1 Partition: 1 Leader: 2 Replicas: 2,4,1,3 Isr: 2,4 Elr: N/A LastKnownElr: N/A
Topic: topic1 Partition: 2 Leader: 4 Replicas: 4,1,3,2 Isr: 4,2 Elr: N/A LastKnownElr: N/A
Topic: topic1 Partition: 3 Leader: 2 Replicas: 1,3,2,4 Isr: 2,4 Elr: N/A LastKnownElr: N/A

# 正常生产
[root@localhost ~]# /opt/kafka_2.13-3.9.0/bin/kafka-console-producer.sh --bootstrap-server 192.168.150.132:9092,192.168.150.129:9092 --topic topic1
>new o^Hbo thatis
>129 failed, 130 became controller
>131 failed,^H 130 controller
>128^H9
>128
>129 and 130

# 正常消费
[root@localhost ~]# /opt/kafka_2.13-3.9.0/bin/kafka-console-consumer.sh --bootstrap-server 192.168.150.132:9092,192.168.150.131:9092,192.168.150.130:9092,192.168.150.129:9092 --topic topic1

129 failed, 130 became controller
131 failed 130 controller
129
128
129 and 131

模拟故障IDC恢复

129和131恢复后
[zk: 192.168.150.128:2181(CONNECTED) 28] ls /brokers/ids
[1, 2, 3, 4]
[zk: 192.168.150.128:2181(CONNECTED) 29] get /controller
{"version":2,"brokerid":2,"timestamp":"1754723996337","kraftControllerEpoch":-1}
虽然ISR都恢复了,但是Leader不会重新选举。
[root@localhost ~]# /opt/kafka_2.13-3.9.0/bin/kafka-topics.sh --bootstrap-server 192.168.150.130:9092 --describe --topic topic1
Topic: topic1 TopicId: P2nKvNOtTLe-ykL7mVcsJw PartitionCount: 4 ReplicationFactor: 4 Configs: min.insync.replicas=2
Topic: topic1 Partition: 0 Leader: 2 Replicas: 3,2,4,1 Isr: 2,4,1,3 Elr: N/A LastKnownElr: N/A
Topic: topic1 Partition: 1 Leader: 2 Replicas: 2,4,1,3 Isr: 2,4,1,3 Elr: N/A LastKnownElr: N/A
Topic: topic1 Partition: 2 Leader: 4 Replicas: 4,1,3,2 Isr: 4,2,1,3 Elr: N/A LastKnownElr: N/A
Topic: topic1 Partition: 3 Leader: 2 Replicas: 1,3,2,4 Isr: 2,4,1,3 Elr: N/A LastKnownElr: N/A
# 正常消费,而且最开始故障的129(broker.id=1)内副本数据已经正常同步了。
129 failed ...和 131 failed就是129和131故障后新生产的数据。
[root@localhost ~]# /opt/kafka_2.13-3.9.0/bin/kafka-console-consumer.sh --bootstrap-server 192.168.150.129:9092 --topictopic1 --from-beginning
third msg
forth message
five messge
go
good moon
let's go
ai
new bo thatis
129 failed, 130 became controller
131 failed 130 controller
hello kfc
this is from kafka

总结

经过验证,证明采用这种方案可以实现跨2AZ部署。

【版权声明】本文为华为云社区用户原创内容,未经允许不得转载,如需转载请自行联系原作者进行授权。如果您发现本社区中有涉嫌抄袭的内容,欢迎发送邮件进行举报,并提供相关证据,一经查实,本社区将立刻删除涉嫌侵权内容,举报邮箱: cloudbbs@huaweicloud.com
  • 点赞
  • 收藏
  • 关注作者

评论(0

0/1000
抱歉,系统识别当前为高风险访问,暂不支持该操作

全部回复

上滑加载中

设置昵称

在此一键设置昵称,即可参与社区互动!

*长度不超过10个汉字或20个英文字符,设置后3个月内不可修改。

*长度不超过10个汉字或20个英文字符,设置后3个月内不可修改。