Hi all, I'm evaluating <https://phabricator.wikimedia.org/T307944> the feasibility of setting up a cross datacenter Kafka 'stretch' cluster at The Wikimedia Foundation.
I've found docs here and there, but they are pretty slim. My biggest concern is the fact that while Follower Fetching <https://cwiki.apache.org/confluence/display/KAFKA/KIP-392%3A+Allow+consumers+to+fetch+from+closest+replica> helps with potential consumer latency in a stretch cluster, there is nothing that addresses producer latency. I'd have expected the docs I've read to mention this if it was a concern, but I haven't seen it. Specifically, let's say I'm a producer in DC-A, and I want to produce to partition X with acks=all. Partition X has 3 replicas, on brokers B1 in DC A, B2 in DC-A and B3 in DC-B. Currently, the replica on B3(DC-B) is the partition leader. IIUC, when I produce my message to partition X, that message will cross the DC boundary for my produce request to B3(DC-B), then back again when replica B1(DC-A) fetches, and also when replica B2(DC-A) fetches, for a total of 3 times between DCs. Questions: - Am I correct in understanding that each one of these fetches contributes to the ack latency? - And, as the number of brokers and replica increases, the number of times a message crosses the DC (likely) increases too? - When replicas are promoted to be a partition leader, producer clients will shuffle their connections around, often resulting in them connecting to the leader in a remote datacenter. Should I be worried about this unpredictability in cross DC network connections and traffic? I'm really hoping that a stretch cluster will help solve some Multi DC streaming app architecture woes, but I'm not so sure the potential issues with partition leaders is worth it! Thanks for any insight y'all have, -Andrew Otto Wikimedia Foundation