Just to make sure I am following correctly, we do not see issues with 1
topic 20k partitions. With 25 brokers, the load is fairly distributed and
we can achieve aggregate throughput of ~2.5 GBps.

With 20k topics with 1 partition each, we see much reduced throughput.

Thanks,
Amit


On Fri, Jul 5, 2024 at 12:08 AM Brebner, Paul
<paul.breb...@netapp.com.invalid> wrote:

> OK so repeating with Java Kafka producer there is no problem – it’s
> specific to the Kafka CLI Producer! Paul
>
> From: Brebner, Paul <paul.breb...@netapp.com.INVALID>
> Date: Friday, 5 July 2024 at 1:21 PM
> To: users@kafka.apache.org <users@kafka.apache.org>
> Subject: Re: Kafka 20k topics metadata update taking long time
> EXTERNAL EMAIL - USE CAUTION when clicking links or attachments
>
>
> Repeating my tests today with a bit more caution I can get up to around
> 47,000 partitions for a single topic before the producer fails with a
> bootstrap broker disconnected warning (in practice the producer cannot
> send), here’s a graph of the producer time (to send 1k messages) using
> producer CLI with increasing partitions – it blows up near the end.  No
> error logs on Kafka brokers or controllers, and everything else still works
> – i.e. can still increase partitions on the big partition topic, can still
> produce/consume on another topic with 3 partitions. RF=3 and 3x4 core
> brokers and 3 dedicated Kraft controllers.
>
> Curious if there is a time out setting somewhere for the client meta-data
> request? Paul
>
> [cid:image001.png@01DACEDE.1DAE3A70]
>
>
> From: Brebner, Paul <paul.breb...@netapp.com.INVALID>
> Date: Thursday, 4 July 2024 at 3:44 PM
> To: users@kafka.apache.org <users@kafka.apache.org>
> Subject: Re: Kafka 20k topics metadata update taking long time
> EXTERNAL EMAIL - USE CAUTION when clicking links or attachments
>
>
>
>
> Hi – interesting, I had maybe similar problems today when “testing” the
> limits of a Kafka cluster for max partitions – I could create a topic with
> lots of partitions (ok so more than sensible, taking into account RF=3 over
> 1M partitions) – but trying to send a message failed with a meta-data
> timeout.
>
> I forgot to test message send/receive with lower partitions, unfortunately.
>
> But, other  topics on the same cluster with normal numbers of partitions
> still worked ok with the large partition topic, Paul
>
> From: Sabarish Sasidharan <sabarish....@gmail.com>
> Date: Thursday, 4 July 2024 at 3:28 PM
> To: users@kafka.apache.org <users@kafka.apache.org>
> Subject: Re: Kafka 20k topics metadata update taking long time
> [You don't often get email from sabarish....@gmail.com. Learn why this is
> important at https://aka.ms/LearnAboutSenderIdentification ]
>
> EXTERNAL EMAIL - USE CAUTION when clicking links or attachments
>
>
>
>
> This is interesting. But 20K topics is also not a realistic assumption to
> have. I don't see an alarming difference in the latency results from the
> two scenarios. Also even when cached, the metadata needs to be refreshed
> anyway.
>
> On Thu, Jul 4, 2024 at 4:37 AM Amit Chopra <amit.cho...@broadcom.com
> .invalid>
> wrote:
>
> > Hey,
> >
> > I wanted to raise a performance issue that I encountered while testing
> on a
> > large scale with Kafka. The issue pertains to low throughput with 20k
> > topics (each topic with 1 partition) compared to good throughput when
> using
> > 1 topic with 20k partitions.
> >
> > Scenario: I have a setup of kafka with 20k partitions with replication
> > factor of 3 and min.insync.replica of 2. We have 25 broker nodes and
> topics
> > are evenly distributed across the brokers. Each broker has 8 cores, 32 GB
> > memory and persistent HDD. I am testing with 1 producer node, writing a
> > record of size 3KB randomly to one of the 20k topics.
> >
> > I have modified the kafka perf producer code to write randomly to
> different
> > topic for each record. And have added additional print statements (as
> > mentioned below).
> >
> > I did some analysis of kafka producer code and found the reason for bad
> > performance.
> >
> > TLDR: poor performance is due to metadata fetch required for each topic.
> In
> > case of 1 topic 20k partitions, it is only required once and is cached.
> In
> > case of 20k topics, metadata is fetched for each topic. And thus takes a
> > ramp up of about ~5 mins to start getting better throughput.
> >
> > Details:
> > 1. I added timers in kafka producer code to identify where time was being
> > spent.
> > 2. Essentially, the producer calls producer.send()
> > <
> >
> https://github.com/apache/kafka/blob/trunk/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java#L119
> <
> https://github.com/apache/kafka/blob/trunk/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java#L119
> ><
> https://github.com/apache/kafka/blob/trunk/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java#L119
> >
> > >.
> > Which in turn makes calls to fetch the metadata from cache. If not
> present,
> > it calls the broker for metadata and waits for response inside
> awaitUpdate
> > <
> >
> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java#L1176
> <
> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java#L1176
> ><
> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java#L1176
> >
> > >
> > 3. this awaitUpdate is causing the poor performance
> >
> > Test Runs - printing every one second, average, max and number of times
> > method is called
> > 1. 1 Topic 20k partitions
> >
> > I have no
> > name!@kafka-producer-bitnami-74847c7dff-h5dbq:/opt/bitnami/kafka/bin$
> > ./kafka-producer-perf-test.sh --topic amit-1 --num-records 100000000
> > --record-size 3000 --throughput 0 --producer-props acks=all
> > bootstrap.servers=test-cluster-kafka-bootstrap:9092 batch.size=3000
> > buffer.memory=100000000 linger.ms=50 | grep producer.send
> > producer.send(): Avg - 0 ms, Max - 478 ms, num_records = 3838
> > producer.send(): Avg - 0 ms, Max - 44 ms, num_records = 3273
> > producer.send(): Avg - 0 ms, Max - 43 ms, num_records = 2574
> > producer.send(): Avg - 0 ms, Max - 45 ms, num_records = 2922
> > producer.send(): Avg - 0 ms, Max - 36 ms, num_records = 3086
> > producer.send(): Avg - 0 ms, Max - 37 ms, num_records = 3014
> >
> >
> > 2. 20k topics
> >
> > I have no
> > name!@kafka-producer-bitnami-74847c7dff-h5dbq:/opt/bitnami/kafka/bin$
> > ./kafka-producer-perf-test.sh --topic test-1 --num-records 100000000
> > --record-size 3000 --throughput 0 --producer-props acks=all
> > bootstrap.servers=test-cluster-kafka-bootstrap:9092 batch.size=3000
> > buffer.memory=100000000 linger.ms=50 | grep producer.send
> > producer.send(): Avg - 5 ms, Max - 236 ms, num_records = 188
> > producer.send(): Avg - 1 ms, Max - 20 ms, num_records = 498
> > producer.send(): Avg - 2 ms, Max - 23 ms, num_records = 438
> > producer.send(): Avg - 2 ms, Max - 16 ms, num_records = 425
> > producer.send(): Avg - 2 ms, Max - 17 ms, num_records = 343
> > producer.send(): Avg - 2 ms, Max - 15 ms, num_records = 334
> > producer.send(): Avg - 3 ms, Max - 15 ms, num_records = 295
> > producer.send(): Avg - 3 ms, Max - 20 ms, num_records = 268
> >
> >
> > Where:
> > Avg - average time taken in 1 second
> > Max - max time taken in the call within 1 sec
> > num_records - number of times the send was called
> >
> > FYI - the average time keeps increasing to around 12 ms in case of 20k
> > topics. In case of 1 topic, it is 0 due to cache return
> >
> > I wanted to check if there could be a capability to fetch metadata for
> all
> > topics (or matching a pattern) at the very beginning. Just like it is
> done
> > for 1 topic with 20k partitions.
> >
> > Thanks,
> > Amit
> >
> > --
> > This electronic communication and the information and any files
> > transmitted
> > with it, or attached to it, are confidential and are intended solely for
> > the use of the individual or entity to whom it is addressed and may
> > contain
> > information that is confidential, legally privileged, protected by
> privacy
> > laws, or otherwise restricted from disclosure to anyone else. If you are
> > not the intended recipient or the person responsible for delivering the
> > e-mail to the intended recipient, you are hereby notified that any use,
> > copying, distributing, dissemination, forwarding, printing, or copying of
> > this e-mail is strictly prohibited. If you received this e-mail in error,
> > please return the e-mail to the sender, delete it from your computer, and
> > destroy any printed copy of it.
> >
>

-- 
This electronic communication and the information and any files transmitted 
with it, or attached to it, are confidential and are intended solely for 
the use of the individual or entity to whom it is addressed and may contain 
information that is confidential, legally privileged, protected by privacy 
laws, or otherwise restricted from disclosure to anyone else. If you are 
not the intended recipient or the person responsible for delivering the 
e-mail to the intended recipient, you are hereby notified that any use, 
copying, distributing, dissemination, forwarding, printing, or copying of 
this e-mail is strictly prohibited. If you received this e-mail in error, 
please return the e-mail to the sender, delete it from your computer, and 
destroy any printed copy of it.

Reply via email to