Page MenuHomePhabricator

Establish a proper process for replacing kafka nodes
Closed, ResolvedPublic

Description

Replacing the kafka-main2001 (T363210) with kafka-mein2006 created an eventgate-main outage, the process used was the following:

Replacement plan (for each node in the cluster):

  • Add new kafka node IPs to the list of IPs in kafka_brokers_main (hieradata/common.yaml)
  • Run puppet on conf nodes (update zookeeper firewall with the new IPs)
  • Ensure that Under Replicated Partitions is 0
  • Downtime new and old node
  • Silence KafkaUnderReplicatedPartitions for the cluster
  • Stop kafka and kafka-mirror and disable puppet on the old broker:

sudo cumin kafka-main2001.codfw.wmnet 'disable-puppet "Hardware refresh - T363210"; systemctl stop kafka-mirror.service kafka.service'

  • Assign the kafka-id of the old node to the new node in hieradata/common.yaml, assign kafka::main role to new node
  • Run puppet on the new node
  • Run puppet on deploy host and deply external-services update to all k8s clusters
  • Roll restart kafka on all other brokers of the cluster (from sre.kafka.roll-restart-reboot-brokers) to read updated config:

/usr/local/bin/kafka-broker-in-sync && systemctl restart kafka && source /etc/profile.d/kafka.sh; kafka preferred-replica-election

We concluded that there are three things that could be improved in the process above:

  1. Throttle the bandwidth used to resync the replaced broker to not saturate it's link
  2. Remove the leadership for all topics from the to be replaced broker before stopping kafka there (and restore after is has fully synced up)
  3. Don't run kafka preferred-replica-election during the roll-restart of the remaining kafka brokers
    1. Bonus points for not having to restart kafka on the remaining brokers at all
      • ssl.principal.mapping.rules is kafka >=2.4, so it won't work.
      • It seems to me like we're using the same cert for authentication and encrypting traffic, thus creating a certificate with the same CN on all brokers and just have that one CN listed in super.users is not an option as well (but I absolutely don't understand this good enough).
  4. In order to avoid resync load on the kafka cluster we could evaluate if it is possible to rsync the data from old node to new node before making the new node a kafka broker (but after stopping kafka on the old node). That would saturate the NICs of those two nodes for some time but while they are not in service. Catching up should be fast then when the new node joins the cluster.

Regarding 1.) we already tested KIP-73 Replication Quotas, namely follower.replication.throttled.replicas and follower.replication.throttled.rate on kafka-jumbo without success.

# Add an ACL allowing us to alter the cluster config
kafka acls --add --allow-host '*' --cluster --operation AlterConfigs --allow-principal User:ANONYMOUS
# Set leader.replication.throttled.replicas for broker-id 1015
kafka-configs --bootstrap-server $KAFKA_BOOTSTRAP_SERVERS --entity-type brokers --entity-name 1015 --alter \
  --add-config 'follower.replication.throttled.rate=300000000'
# This change was accepted, but not reflected in kafka-config --bootstrap-server $KAFKA_BOOTSTRAP_SERVERS \
  --entity-type brokers --entity-name 1015 --describe
# but it was in kafka configs --describe --entity-type brokers | grep 1015
# Setting both, follower.replication.throttled.rate and follower.replication.throttled.replicas also had no visible effect
kafka-configs --bootstrap-server $KAFKA_BOOTSTRAP_SERVERS --entity-type brokers --entity-name 1015  --alter \
  --add-config 'follower.replication.throttled.replicas=1015,follower.replication.throttled.rate=2000000'
# Settings could be deleted via
kafka-configs --bootstrap-server $KAFKA_BOOTSTRAP_SERVERS --entity-type brokers --entity-name 1015 --alter \
  --delete-config 'follower.replication.throttled.replicas,follower.replication.throttled.rate'

Further reads:

Event Timeline

I'll add some thoughts as well.

In our testing, we haven't been able to throttle the follower replication traffic for a given broken in the absence of a reassignment. It would be convenient to be able to do so, but for now this eludes us.
This reinforces the impact of evacuating leadership of the broker aimed for replacement beforehand. This way, when the new hardware with empty disk and the same broker id comes back, it will catch up unthrottled, but will only be a follower for each assigned partition. The inbound traffic should not backpressure to the producers/consumers.

Evacuating leadership for a broker (1015 in that example) can be done this way:

brouberol@kafka-jumbo1015:~$ topicmappr rebuild --leader-evac-brokers 1015 --leader-evac-topics '.*'  --brokers -1 --topics '.*' --out-file evacuate-1015-leadership --skip-no-ops

This creates a evacuate-1015-leadership.json file that will remove leadership for all partitions currently assigned to broker 1015, and is applied the usual way:

brouberol@kafka-jumbo1015:~$ kafka-reassign-partitions --reassignment-json-file evacuate-1015-leadership.json --execute
NOTE: this is only changing leadership. No data will be moved.

The command will also output the previous state, as json. Save this to a evacuate-1015-leadership-rollback.json file. Once the new broker has fully caught up, you can simply rollback the partition leadership state by running:

brouberol@kafka-jumbo1015:~$ kafka-reassign-partitions --reassignment-json-file evacuate-1015-leadership-rollback.json --execute

Just FTR -- we didn't actually saturate anything at all on the Kafka hosts. The hottest that any NIC was was about 70% of line rate, which is warm but not hot. No drops or errors on the switch ports, nic-saturation-exporter metrics show that we weren't micro-bursting above 800mbit/s except for a one-minute interval (and nic-sat says were below 900mbit/s during then). Disk i/o queues were fine, all the other NICs were fine, and it wasn't some goofy thing where we were saturating core 0 or anything on kafka-main2006 either (link).

I think the only real problem here was the EventGate readiness probe.

Change #1071559 had a related patch set uploaded (by JMeybohm; author: JMeybohm):

[operations/cookbooks@master] kafka/roll-restart-reboot-brokers: Add exclude and no-election options

https://gerrit.wikimedia.org/r/1071559

Change #1071559 merged by jenkins-bot:

[operations/cookbooks@master] kafka/roll-restart-reboot-brokers: Add exclude and no-election options

https://gerrit.wikimedia.org/r/1071559

During the current replacement with the above process (replacing broker id 2002, kafka-main2002 -> kafka-main2007) @Vgutierrez reported errors from purged:

Sep 10 10:25:09 cp2032 purged[836318]: %4|1725963909.573|SESSTMOUT|purged#consumer-1| [thrd:main]: Consumer group session timed out (in join-state steady) after 10377 ms without a successful response from the group coordinator (broker 2001, last error was Local: Broker transport failure): revoking assignment and rejoining group
Sep 10 10:28:34 cp2032 purged[836318]: %4|1725964114.276|SESSTMOUT|purged#consumer-1| [thrd:main]: Consumer group session timed out (in join-state steady) after 10078 ms without a successful response from the group coordinator (broker 2001, last error was Local: Broker transport failure): revoking assignment and rejoining group
Sep 10 10:29:14 cp2032 purged[836318]: %4|1725964154.815|SESSTMOUT|purged#consumer-1| [thrd:main]: Consumer group session timed out (in join-state steady) after 10213 ms without a successful response from the group coordinator (broker 2001, last error was Local: Broker transport failure): revoking assignment and rejoining group
Sep 10 10:30:24 cp2032 purged[836318]: %4|1725964224.486|SESSTMOUT|purged#consumer-1| [thrd:main]: Consumer group session timed out (in join-state steady) after 10245 ms without a successful response from the group coordinator (broker 2001, last error was Local: Broker transport failure): revoking assignment and rejoining group

Change #1071841 had a related patch set uploaded (by Vgutierrez; author: Vgutierrez):

[operations/puppet@production] hiera: let purged@codfw|ulsfo use main-eqiad kafka cluster

https://gerrit.wikimedia.org/r/1071841

Change #1071841 merged by Vgutierrez:

[operations/puppet@production] hiera: let purged@codfw|ulsfo use main-eqiad kafka cluster

https://gerrit.wikimedia.org/r/1071841

Mentioned in SAL (#wikimedia-operations) [2024-09-10T10:51:48Z] <vgutierrez> switching purged in codfw and ulsfo to use main-eqiad kafka cluster - T373189

We where this time seeing huge spikes in RTT from eventgate-main to some kafka brokers (mainly kafka-main2006 over a long period) that propagated down to high latency of POST's to eventgate-main - saturating fpm workers of mediawiki: https://grafana-rw.wikimedia.org/d/ZB39Izmnz/eventgate?orgId=1&var-service=eventgate-main&var-stream=All&var-kafka_broker=All&var-kafka_producer_type=All&var-dc=thanos&var-site=codfw&from=1725956972834&to=1725968816809

Last time we did not see those because eventgate-main killed itself (T373192), but this is still problematic. Especially since the high latency period lasted ~60min.

I don't see why kafka-main2006 was responding so slow. It was under more load than usual (naturally), but far from overloaded.

JMeybohm claimed this task.

Given the last replacement did again cause service disruption (even though less bad than the first replacement) I went with copying all kafka files manually before starting up kafka to keep the resync time short. This seems to do the trick, keeping the resync time in the area of minutes rather than hours. I still see some elevated latency in eventgate-main, but nothing compared to the last two replacements.

The process is now described at https://wikitech.wikimedia.org/wiki/Kafka/Administration#Hardware_replace_a_broker

jijiki renamed this task from Establish a proper process for repacing kafka nodes to Establish a proper process for replacing kafka nodes.Nov 11 2024, 5:07 PM