Following up to check if someone can help. And is this the right approach or should i open a Jira ticket for same.
Thanks, Amit On Fri, Jul 5, 2024 at 9:42 AM Amit Chopra <amit.cho...@broadcom.com> wrote: > But 20K topics is also not a realistic assumption to have. > This is our existing scale with AWS kinesis 20k shards. And we are moving > over to kafka and thus testing with equivalent scale. We are looking at an > ingestion rate of 2.5 GBps. > > I don't see an alarming difference in the latency results from the two > scenarios. > The output I shared before was showing the latency being introduced per > record being sent due to metadata lookup. It is about 3-5 ms per record. > The main aspect is that it causes only ~300 records to be sent per second > with 20k topics. While can send ~3000 records per second with 1 topic (20k > partitions). > > Also sharing the throughput difference in test results. > > Test - 1 topic with 20k partitions > > ./kafka-producer-perf-test.sh --topic amit-1 --num-records 100000000 > --record-size 3000 --throughput -1 --producer-props acks=all > bootstrap.servers=test-cluster-kafka-bootstrap:9092 batch.size=3000 > buffer.memory=100000000 linger.ms=50 > > 14897 records sent, 2964.0 records/sec (8.48 MB/sec), 131.7 ms avg > latency, 1127.0 ms max latency. > > 138132 records sent, 27407.1 records/sec (78.41 MB/sec), 246.2 ms avg > latency, 1529.0 ms max latency. > > 219604 records sent, 43365.7 records/sec (124.07 MB/sec), 743.2 ms avg > latency, 3432.0 ms max latency. > > 221991 records sent, 44389.3 records/sec (127.00 MB/sec), 711.1 ms avg > latency, 3011.0 ms max latency. > > 225060 records sent, 44823.7 records/sec (128.24 MB/sec), 685.9 ms avg > latency, 2631.0 ms max latency. > > 208351 records sent, 40989.8 records/sec (117.27 MB/sec), 733.4 ms avg > latency, 2949.0 ms max latency. > > > > Test - 20k topics with 1 partition each > > ./kafka-producer-perf-test.sh --topic test1-1 --num-records 100000000 > --record-size 3000 --throughput -1 --producer-props acks=all > bootstrap.servers=test-cluster-kafka-bootstrap:9092 batch.size=3000 > buffer.memory=100000000 linger.ms=50 > > Throughput: {}0 > > 1962 records sent, 392.3 records/sec (1.12 MB/sec), 48.1 ms avg latency, > 449.0 ms max latency. > > 1382 records sent, 275.8 records/sec (0.79 MB/sec), 50.7 ms avg latency, > 77.0 ms max latency. > > 1011 records sent, 201.6 records/sec (0.58 MB/sec), 56.2 ms avg latency, > 82.0 ms max latency. > > 886 records sent, 176.7 records/sec (0.51 MB/sec), 58.8 ms avg latency, > 91.0 ms max latency. > > 714 records sent, 142.0 records/sec (0.41 MB/sec), 87.5 ms avg latency, > 241.0 ms max latency. > > 687 records sent, 137.4 records/sec (0.39 MB/sec), 79.6 ms avg latency, > 202.0 ms max latency. > > > > > Also even when cached, the metadata needs to be refreshed anyway. > We were hoping to keep the cache metadata for a much longer duration. So > that there is no need for frequent refresh > > Thanks, > Amit > > On Thu, Jul 4, 2024 at 12:30 AM Sabarish Sasidharan < > sabarish....@gmail.com> wrote: > >> 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 >> > >. >> > 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 >> > > >> > 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.