Hi,

two questions. Is your MirrorMaker collocated with the source or the target?
what are the send and receive buffer sizes on the connections that do span
across WAN?

Hope we can get you some help.

Best jan



On 06.12.2017 14:36, Xu, Zhaohui wrote:
Any update on this issue?

We also run into similar situation recently. The mirrormaker is leveraged to 
replicate messages between clusters in different dc. But sometimes a portion of 
partitions are with high consumer lag and tcpdump also shows similar packet 
delivery pattern. The behavior is sort of weird and is not self-explaining. 
Wondering whether it has anything to do with the fact that number of consumers 
is too large?  In our example, we have around 100 consumer connections per 
broker.
        
Regards,
Jeff

On 12/5/17, 10:14 AM, "tao xiao" <xiaotao...@gmail.com> wrote:

     Hi,
any pointer will be highly appreciated On Thu, 30 Nov 2017 at 14:56 tao xiao <xiaotao...@gmail.com> wrote: > Hi There,
     >
     >
     >
     > We are running into a weird situation when using Mirrormaker to replicate
     > messages between Kafka clusters across datacenter and reach you for help 
in
     > case you also encountered this kind of problem before or have some 
insights
     > in this kind of issue.
     >
     >
     >
     > Here is the scenario. We have setup a deployment where we run 30
     > Mirrormaker instances on 30 different nodes. Each Mirrormaker instance is
     > configure with num.streams=1 thus only one consumer runs. The topics to
     > replicate is configure with 100 partitions and data is almost evenly
     > distributed across all partitions. After running a period of time, weird
     > things happened that some of the Mirrormaker instances seems to slow down
     > and consume at a relative slow speed from source Kafka cluster. The 
output
     > of tcptrack shows the consume rate of problematic instances dropped to
     > ~1MB/s, while the other healthy instances consume at a rate of  ~3MB/s. 
As
     > a result, the consumer lag for corresponding partitions are going high.
     >
     >
     >
     >
     > After triggering a tcpdump, we noticed the traffic pattern in tcp
     > connection of problematic Mirrmaker instances is very different from
     > others. Packets flowing in problematic tcp connections are relatively 
small
     > and seq and ack packets are basically coming in one after another. On the
     > other hand, the packets in healthy tcp connections are coming in a
     > different pattern, basically several seq packets comes with an ack 
packets.
     > Below screenshot shows the situation, and these two captures are got on 
the
     > same mirrormaker node.
     >
     >
     >
     > problematic connection.  ps. 10.kfk.kfk.kfk is kafka broker, 10.mm.mm.mm
     > is Mirrormaker node
     >
     > 
https://na01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fimgur.com%2FZ3odjjT&data=02%7C01%7Czhaohxu%40ebay.com%7Ca8efe84f9feb47ecb5fd08d53b85d7ac%7C46326bff992841a0baca17c16c94ea99%7C0%7C0%7C636480368398154028&sdata=2DdGcjPWD7QI7lZ7v7QDN6I53P9tsSTMzEGdw6IywmU%3D&reserved=0
     >
     >
     > healthy connection
     >
     > 
https://na01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fimgur.com%2Fw0A6qHT&data=02%7C01%7Czhaohxu%40ebay.com%7Ca8efe84f9feb47ecb5fd08d53b85d7ac%7C46326bff992841a0baca17c16c94ea99%7C0%7C0%7C636480368398154028&sdata=v52DmmY9LHN2%2F59Hb5Xo77JuLreOA3lfDyq8eHKmISQ%3D&reserved=0
     >
     >
     > If we stop the problematic Mirrormaker instance and when other instances
     > take over the lagged partitions, they can consume messages quickly and
     > catch up the lag soon. So the broker in source Kafaka cluster is supposed
     > to be good. But if Mirrormaker itself causes the issue, how can one tcp
     > connection is good but others are problematic since the connections are 
all
     > established in the same manner by Kafka library.
     >
     >
     >
     > Consumer configuration for Mirrormaker instance as below.
     >
     > auto.offset.reset=earliest
     >
     >
     > 
partition.assignment.strategy=org.apache.kafka.clients.consumer.RoundRobinAssignor
     >
     > heartbeat.interval.ms=10000
     >
     > session.timeout.ms=120000
     >
     > request.timeout.ms=150000
     >
     > receive.buffer.bytes=1048576
     >
     > max.partition.fetch.bytes=2097152
     >
     > fetch.min.bytes=1048576
     >
     >
     >
     > Kafka version is 0.10.0.0 and we have Kafka and Mirrormaker run on Ubuntu
     > 14.04
     >
     >
     >
     > Any response is appreciated.
     >
     > Regards,
     >
     > Tao
     >

Reply via email to