One of the best pieces of advice I can offer is that you really need to run the mirror maker in the same physical/network location as the Kafka cluster you are producing to. Latency on the consumer side can be more easily absorbed than latency on the producer side, as to assure that we have proper message ordering and reliability, we need to restrict in flight batches to 1. So that means that our produce connection is contstrained to be very thin, and latency makes a huge impact. Meanwhile, on the consume side we’re fetching large batches of messages, many at a time, so round-trip latency has less of an impact. I really can’t stress this enough. We set up some mirror makers in the opposite configuration for security reasons, and it’s been a huge problem even with tuning.
In addition to this, you will want to assure that your OS (and then the mirror maker and broker) tuning is taking into account the latency. Here’s a good reference for the OS side (for Linux): http://linuxczar.net/blog/2016/09/18/bandwidth-delay-product/ Once you have the OS tuned, you’ll need to adjust the broker tuning on the clusters you are consuming from, since that is the high latency side. The configuration for that is socket.send.buffer.bytes, and it probably makes sense to set this to -1 (which means use the OS configuration). You can do the same with socket.receive.buffer.bytes, but it’s not as critical with this setup. On the mirror maker, the configuration is on the consumer side, and it’s called receive.buffer.bytes. Again, you can set this to -1 to use the OS configuration. Make sure to restart the applications after making all these changes, of course. -Todd On Sat, Jul 22, 2017 at 1:27 AM, James Cheng <wushuja...@gmail.com> wrote: > Becket Qin from LinkedIn spoke at a meetup about how to tune the Kafka > producer. One scenario that he described was tuning for situations where > you had high network latency. See slides at https://www.slideshare.net/ > mobile/JiangjieQin/producer-performance-tuning-for-apache-kafka-63147600 > and video at https://youtu.be/oQe7PpDDdzA > > -James > > Sent from my iPhone > > > On Jul 21, 2017, at 9:25 AM, Sunil Parmar <sunilosu...@gmail.com> wrote: > > > > We're trying to set up mirror maker to mirror data from EU dc to US dc. > The > > network delay is ~150 ms. In recent test; we realized that mirror maker > is > > not keeping up with load and have a lag trending upward all the time. > > > > What are configurations that can be tuned up to make it work for the > higher > > throughput ? > > How to decide number of producer and consumer threads ? ( number of topic > > partitions / brokers ? ) > > > > > > *Environment* ( both source and destination cluster ) > > > > Kafka version 0.9 ( Cloudera 0.9.0.0+kafka2.0.0+188 ) > > > > queue.size = 10000 > > queue.byte.size = 100MB > > > > 2 brokers on source, 3 brokers on destination > > > > > > *Mirror maker configs :* > > > > Producer properties : > > request.timeout.ms=120000 > > block.on.buffer.full=TRUE > > retries=2000000000 > > acks=all > > > > > > Consumer properties: > > request.timeout.ms=120000 > > auto.offset.reset=latest > > enable.auto.commit=false > > > > We've configured 4 producer and consumer threads. > > There is no security set up as of now so it's all PLAINTEXT. > > > > We have 4 topics are white listed to sync from EU to US. Only one of them > > is high throughput. We also have a message handler to strip off some > > sensitive information from EU to US but it only works on a low thru put > > topic; the message handler still try to process the other topics but let > it > > pass thru. > > > > Thanks, > > Sunil Parmar > -- *Todd Palino* Senior Staff Engineer, Site Reliability Data Infrastructure Streaming linkedin.com/in/toddpalino