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

ASF GitHub Bot commented on KAFKA-8904:
---------------------------------------

bdbyrne commented on pull request #7781: KAFKA-8904: Improve producer's topic 
metadata fetching.
URL: https://github.com/apache/kafka/pull/7781
 
 
   Improves the producer's topic metadata fetching logic by maintaining a 
per-topic last refresh time, which is used to discriminate which topics' 
metadata should be fetched.
   
   The improvement can be witnessed during producer startup, where many topics 
may be fetched serially. Previously, where N topics were being fetched 
serially, O(N^2) total topic metadata would be processed, whereas this 
improvement makes it O(N).
   
   A simple test can show the differences fairly easily. Starting a producer on 
500 existing topics with 64 partitions each, single-threaded:
   
   Old: 1000 records sent, 32.446463 records/sec (0.00 MB/sec), 65.52 ms avg 
latency, 459.00 ms max latency, 23 ms 50th, 213 ms 95th, 360 ms 99th, 459 ms 
99.9th.
   
   New: 1000 records sent, 206.143063 records/sec (0.00 MB/sec), 10.55 ms avg 
latency, 187.00 ms max latency, 7 ms 50th, 29 ms 95th, 43 ms 99th, 187 ms 
99.9th.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Reduce metadata lookups when producing to a large number of topics
> ------------------------------------------------------------------
>
>                 Key: KAFKA-8904
>                 URL: https://issues.apache.org/jira/browse/KAFKA-8904
>             Project: Kafka
>          Issue Type: Improvement
>          Components: controller, producer 
>            Reporter: Brian Byrne
>            Priority: Minor
>
> Per [~lbradstreet]:
>  
> "The problem was that the producer starts with no knowledge of topic 
> metadata. So they start the producer up, and then they start sending messages 
> to any of the thousands of topics that exist. Each time a message is sent to 
> a new topic, it'll trigger a metadata request if the producer doesn't know 
> about it. These metadata requests are done in serial such that if you send 
> 2000 messages to 2000 topics, it will trigger 2000 new metadata requests.
>  
> Each successive metadata request will include every topic seen so far, so the 
> first metadata request will include 1 topic, the second will include 2 
> topics, etc.
>  
> An additional problem is that this can take a while, and metadata expiry (for 
> metadata that has not been recently used) is hard coded to 5 mins, so if this 
> the initial fetches take long enough you can end up evicting the metadata 
> before you send another message to a topic.
> So the approaches above are:
> 1. We can linger for a bit before making a metadata request, allow more sends 
> to go through, and then batch the metadata request for topics we we need in a 
> single metadata request.
> 2. We can allow pre-seeding the producer with metadata for a list of topics 
> you care about.
> I prefer 1 if we can make it work."



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to