Hello Andrew. Just to answer your questions first, yes that's correct in your described settings that three round-trips between DCs would incur, but since the replica fetches can be done in parallel, the latency is not a sum of all the round-trips. But if you stay with 2 DCs only, the number of round-trips would only depend on the number of follower replicas that are on different DCs with the leader replica.
Jumping out of the question and your described settings, there are a couple of things you can consider for your design: 1) For the producer -> leader hop, could you save the cross-DC network? For example, if your message can potentially go to any partitions (such as it is not key-ed), then you can customize your partitioner as a "rack-aware" one that would always try to pick the partition whose leader co-exist within the same DC as the producer client; even if your message's partition has to be determined deterministically by the key, in operations you can still see if most of your active producers are from one DC, then configure your topic partitions to be hosted by brokers within the same DC. Generally speaking, there are various ways you can consider saving this hop from across DCs. 2) For the leader -> follower hop, you can start from first validating how many failures cross DCs that you'd like to tolerate. For example, let's say you have 2N+1 replicas per partition, with N+1 replicas including the leader on one DC and N other replicas on the other DC, if we can set the acks to N+2 then it means we will have the data replicated at least on one remote replica before returning the request, and hence the data would not be lost if the one whole DC fails, which could be sufficient from many stretching and multi-colo cases. Then in practice, since the cross-colo usually takes more latency, you'd usually get much fewer round-trips than N across DC before satisfying the acks. And your average/p99 latencies would not increase much compared with just one cross-DC replica. Guozhang On Mon, May 9, 2022 at 11:58 AM Andrew Otto <o...@wikimedia.org> wrote: > 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 > -- -- Guozhang