[ 
https://issues.apache.org/jira/browse/KAFKA-16344?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17830685#comment-17830685
 ] 

Greg Harris edited comment on KAFKA-16344 at 3/25/24 8:06 PM:
--------------------------------------------------------------

Hi [~janardhanag] Yes, you should consider increasing offset.lag.max. 0 is best 
for precise offset translation, but can significantly increase the amount of 
traffic on the offset-syncs topic.

For topics without offset gaps (e.g. not compacted, not transactional) the 
offset.lag.max currently behaves as a ratio between the amount of records/sec 
in a mirrored topic, and that topic's additional load on the offset-syncs 
topic. There's a semaphore 
[https://github.com/apache/kafka/blob/f8ce7feebcede6c6da93c649413a64695bc8d4c1/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java#L95]
 that prevents this load from slowing down the mirror source connector, but it 
won't protect the brokers holding the mm2 offsets topic from being overloaded.

For example, if you were to mirror 1000000 records, offset.lag.max=0 could 
write 1000000 records to the offset-syncs topic, generating huge load. For 
1000000 records with offset.lag.max=100, the number of offset syncs could be 
10000, 100x less. For offset.lag.max=10000, the number of offset syncs could be 
100, 10000x less. To answer your questions:

> 1. What is the proportion of mm2-offsetsyncsinternal topic writes to overall 
>MM2 traffic?

Configurable with offset.lag.max for topics with contiguous offsets. For 
transactional topics it isn't configurable, as each transaction may cause 1 or 
2 offset syncs and that isn't limited by the offset.lag.max, so reducing the 
throughput may require increasing the transaction intervals on your source 
applications.

> 2. Is there a way to tune the internal topic writes with increased traffic 
> MM2?

As the throughput of your MM2 instance increases, you should increase the 
offset.lag.max to keep the offset-syncs topic fixed.

> 3. I want to understand the expected mm2-offsetsyncsinternal write TPS for a 
> given amount of MM2 traffic. Are there any tunable parameters to reduce these 
> writes, and what are the consequences of tuning if any?

There isn't a closed formula, because a sync may be initiated by multiple 
conditions 
[https://github.com/apache/kafka/blob/f8ce7feebcede6c6da93c649413a64695bc8d4c1/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java#L360-L363]
 and prevented by the semaphore when the producer latency becomes significant.

The consequence of decreasing throughput on this topic is less precise offset 
translation in the MirrorCheckpointConnector, and increased redelivery when 
failing over from source to target consumer offsets. If you aren't using this 
connector, then the offset syncs topic isn't read from at all. You currently 
can't turn the topic off, but there is a KIP open for that: 
[https://cwiki.apache.org/confluence/display/KAFKA/KIP-1031%3A+Control+offset+translation+in+MirrorSourceConnector]
 . If for example you were satisfied with 30 seconds of redelivery for a topic 
which had 1000 records/second, then you could set offset.lag.max to 30*1000.

> 4. In a larger Kafka cluster, if a single broker is overloaded with 
> mm2-offsetsyncsinternal traffic, can it lead to a broker crash? Are there any 
> guidelines available for such scenarios? Currently, we have 6 brokers with 
> 24K MM2 traffic, and internal writes are at 10K, resulting in a 20% CPU 
> increase on one broker.

I'm not familiar with the failure conditions for overloaded brokers. If you are 
seeing failures then I would definitely recommend trying to tune MM2, or try to 
use quotas [https://kafka.apache.org/documentation/#design_quotas] to get this 
topic-partition under control.

> 5. Are there any limitations on Kafka brokers scaling , I mean how much kafka 
>broker can be expanded in single Kakfa cluster due to the 
>mm2-offsetsyncsinternal topic?

Because the offset syncs topic only supports a single partition, adding brokers 
will not solve this problem, other than having a broker dedicated to only 
serving this one topic. At some point, you would need two separate offset-syncs 
topics and separate MirrorSourceConnectors to shard the load.

> 6. How can the system be dimensioned to handle MM2 internal topic writes 
> effectively? Are there any recommended figures available? For instance, for a 
> given amount of traffic (X), what percentage increase in CPU (Y) should each 
> broker have to handle MM2 internal topic writes? Note that in other pods, 
> this resource may not be utilized.

The Kafka project doesn't publish "expected" dimensions, other than the default 
values of configurations. We don't publish performance analysis on any 
particular hardware setups, because the diversity of hardware running Kafka is 
just too vast to capture properly.


was (Author: gharris1727):
Hi [~janardhanag] Yes, you should consider increasing offset.lag.max. 0 is best 
for precise offset translation, but can significantly increase the amount of 
traffic on the offset-syncs topic.

For topics without offset gaps (e.g. not compacted, not transactional) the 
offset.lag.max currently behaves as a ratio between the amount of records/sec 
in a mirrored topic, and that topic's additional load on the offset-syncs 
topic. There's a semaphore 
https://github.com/apache/kafka/blob/f8ce7feebcede6c6da93c649413a64695bc8d4c1/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java#L95
 that prevents this load from slowing down the mirror source connector, but it 
won't protect the brokers holding the mm2 offsets topic from being overloaded.

For example, if you were to mirror 1000000 records, offset.lag.max=0 could 
write 1000000 records to the offset-syncs topic, generating huge load. For 
1000000 records with offset.lag.max=100, the number of offset syncs could be 
10000, 100x less. For offset.lag.max=10000, the number of offset syncs could be 
100, 10000x less. To answer your questions:

> 1. What is the proportion of mm2-offsetsyncsinternal topic writes to overall 
>MM2 traffic?

Configurable with offset.lag.max for topics with contiguous offsets. For 
transactional topics it isn't configurable, as each transaction may cause 1 or 
2 offset syncs and that isn't limited by the offset.lag.max, so reducing the 
throughput may require increasing the transaction intervals on your source 
applications.

> 2. Is there a way to tune the internal topic writes with increased traffic 
> MM2?

As the throughput of your MM2 instance increases, you should decrease the 
offset.lag.max to keep the offset-syncs topic fixed.

> 3. I want to understand the expected mm2-offsetsyncsinternal write TPS for a 
> given amount of MM2 traffic. Are there any tunable parameters to reduce these 
> writes, and what are the consequences of tuning if any?

There isn't a closed formula, because a sync may be initiated by multiple 
conditions 
[https://github.com/apache/kafka/blob/f8ce7feebcede6c6da93c649413a64695bc8d4c1/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java#L360-L363]
 and prevented by the semaphore when the producer latency becomes significant.

The consequence of decreasing throughput on this topic is less precise offset 
translation in the MirrorCheckpointConnector, and increased redelivery when 
failing over from source to target consumer offsets. If you aren't using this 
connector, then the offset syncs topic isn't read from at all. You currently 
can't turn the topic off, but there is a KIP open for that: 
[https://cwiki.apache.org/confluence/display/KAFKA/KIP-1031%3A+Control+offset+translation+in+MirrorSourceConnector]
 . If for example you were satisfied with 30 seconds of redelivery for a topic 
which had 1000 records/second, then you could set offset.lag.max to 30*1000.

> 4. In a larger Kafka cluster, if a single broker is overloaded with 
> mm2-offsetsyncsinternal traffic, can it lead to a broker crash? Are there any 
> guidelines available for such scenarios? Currently, we have 6 brokers with 
> 24K MM2 traffic, and internal writes are at 10K, resulting in a 20% CPU 
> increase on one broker.

I'm not familiar with the failure conditions for overloaded brokers. If you are 
seeing failures then I would definitely recommend trying to tune MM2, or try to 
use quotas [https://kafka.apache.org/documentation/#design_quotas] to get this 
topic-partition under control.

> 5. Are there any limitations on Kafka brokers scaling , I mean how much kafka 
>broker can be expanded in single Kakfa cluster due to the 
>mm2-offsetsyncsinternal topic?

Because the offset syncs topic only supports a single partition, adding brokers 
will not solve this problem, other than having a broker dedicated to only 
serving this one topic. At some point, you would need two separate offset-syncs 
topics and separate MirrorSourceConnectors to shard the load.

> 6. How can the system be dimensioned to handle MM2 internal topic writes 
> effectively? Are there any recommended figures available? For instance, for a 
> given amount of traffic (X), what percentage increase in CPU (Y) should each 
> broker have to handle MM2 internal topic writes? Note that in other pods, 
> this resource may not be utilized.

The Kafka project doesn't publish "expected" dimensions, other than the default 
values of configurations. We don't publish performance analysis on any 
particular hardware setups, because the diversity of hardware running Kafka is 
just too vast to capture properly.

> Internal topic mm2-offset-syncs<clustername>internal created with single 
> partition is putting more load on the broker
> ---------------------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-16344
>                 URL: https://issues.apache.org/jira/browse/KAFKA-16344
>             Project: Kafka
>          Issue Type: Bug
>          Components: connect
>    Affects Versions: 3.5.1
>            Reporter: Janardhana Gopalachar
>            Priority: Major
>
> We are using Kafka 3.5.1 version, we see that the internal topic created by 
> mirrormaker 
> mm2-offset-syncs<clustername>internal is created with single partition due to 
> which the CPU load on the broker which will be leader for this partition is 
> increased compared to other brokers. Can multiple partitions be  created for 
> the topic so that the CPU load would get distributed 
>  
> Topic: mm2-offset-syncscluster-ainternal    TopicId: XRvTDbogT8ytNhqX2YTyrA   
>  PartitionCount: 1ReplicationFactor: 3    Configs: 
> min.insync.replicas=2,cleanup.policy=compact,message.format.version=3.0-IV1
>     Topic: mm2-offset-syncscluster-ainternal    Partition: 0    Leader: 2    
> Replicas: 2,1,0    Isr: 2,1,0



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to