Ismael,

I don't see how I can add any information to KAFKA-3358 as I haven't got an 
account?

Phil

-----Original Message-----
From: isma...@gmail.com [mailto:isma...@gmail.com] On Behalf Of Ismael Juma
Sent: 12 April 2016 14:59
To: users@kafka.apache.org
Subject: Re: KafkaProducer 0.9.0.1 continually sends metadata requests

Sorry, I should say that KAFKA-3306 will fix the issue where we request data 
for all topics instead of no topics. However, it seems like there is an 
additional issue where it seems like we are performing metadata requests too 
frequently. Phil, can you please add this information to KAFKA-3358 so that we 
can figure out if more needs to be done?

Ismael

On Tue, Apr 12, 2016 at 2:52 PM, Ismael Juma <ism...@juma.me.uk> wrote:

> Note that this should be fixed as part of
> https://issues.apache.org/jira/browse/KAFKA-3306
>
> Ismael
>
> On Tue, Apr 12, 2016 at 2:17 PM, Phil Luckhurst < 
> phil.luckhu...@encycle.com> wrote:
>
>> Thanks Jonathan, I didn't spot that JIRA.
>>
>> Phil
>>
>> -----Original Message-----
>> From: Jonathan Bond [mailto:jb...@netflix.com.INVALID]
>> Sent: 12 April 2016 14:08
>> To: users@kafka.apache.org
>> Subject: Re: KafkaProducer 0.9.0.1 continually sends metadata 
>> requests
>>
>> Phil,
>> In our case this bug placed significant load on our brokers. We 
>> raised a bug  https://issues.apache.org/jira/browse/KAFKA-3358 to get 
>> this resolved.
>>
>> On Tue, Apr 12, 2016 at 5:39 AM Phil Luckhurst < 
>> phil.luckhu...@encycle.com>
>> wrote:
>>
>> > With debug logging turned on we've sometimes seen our logs filling 
>> > up with the kafka producer sending metadata requests every 100ms e.g.
>> >
>> > 2016-04-08 10:39:33,592 DEBUG [kafka-producer-network-thread | 
>> > phil-pa-1]
>> > org.apache.kafka.clients.NetworkClient: Sending metadata request 
>> > ClientRequest(expectResponse=true, callback=null,
>> > request=RequestSend(header={api_key=3,api_version=0,correlation_id=
>> > 249 ,client_id=phil-pa-1}, 
>> > body={topics=[phil-pa-1-device-update]}),
>> > isInitiatedByNetworkClient, createdTimeMs=1460108373592, 
>> > sendTimeMs=0) to node 0
>> > 2016-04-08 10:39:33,592 DEBUG [kafka-producer-network-thread | 
>> > phil-pa-1]
>> > org.apache.kafka.clients.Metadata: Updated cluster metadata version
>> > 248 to Cluster(nodes = [Node(0, ta-eng-kafka2, 9092)], partitions = 
>> > [Partition(topic = phil-pa-1-device-update, partition = 0, leader = 
>> > 0, replicas = [0,], isr = [0,]])
>> > 2016-04-08 10:39:33,698 DEBUG [kafka-producer-network-thread | 
>> > phil-pa-1]
>> > org.apache.kafka.clients.NetworkClient: Sending metadata request 
>> > ClientRequest(expectResponse=true, callback=null,
>> > request=RequestSend(header={api_key=3,api_version=0,correlation_id=
>> > 250 ,client_id=phil-pa-1}, 
>> > body={topics=[phil-pa-1-device-update]}),
>> > isInitiatedByNetworkClient, createdTimeMs=1460108373698, 
>> > sendTimeMs=0) to node 0
>> > 2016-04-08 10:39:33,698 DEBUG [kafka-producer-network-thread | 
>> > phil-pa-1]
>> > org.apache.kafka.clients.Metadata: Updated cluster metadata version
>> > 249 to Cluster(nodes = [Node(0, ta-eng-kafka2, 9092)], partitions = 
>> > [Partition(topic = phil-pa-1-device-update, partition = 0, leader = 
>> > 0, replicas = [0,], isr = [0,]])
>> >
>> > These metadata requests continue to be sent every 100ms
>> > (retry.backoff.ms) until we stop the process.
>> >
>> > This only seems to happen if the KafkaProducer instance is created 
>> > but not used to publish a message for 5 minutes. After 5 minutes (
>> > metadata.max.age.ms) the producer thread sends a metadata request 
>> > to the server that has an empty topics list and the server responds 
>> > with the partition information for *all* topics hosted on the server.
>> >
>> > 2016-04-11 14:16:39,320 DEBUG [kafka-producer-network-thread | 
>> > phil-pa-1]
>> > org.apache.kafka.clients.NetworkClient: Sending metadata request 
>> > ClientRequest(expectResponse=true, callback=null, 
>> > request=RequestSend(header={api_key=3,api_version=0,correlation_id=
>> > 0,c lient_id=phil-pa-1}, body={topics=[]}), 
>> > isInitiatedByNetworkClient, createdTimeMs=1460380599289,
>> > sendTimeMs=0) to node -1
>> >
>> > If we then use that KafkaProducer instance to send a message the 
>> > next 'Sending meta request' will just be for the topic we have sent 
>> > the message to and this then triggers the flood of retry requests 
>> > as noted
>> above.
>> >
>> > If we ensure we send the first message within the time set by 
>> > metadata.max.age.ms (default 5 minutes) then everything works as 
>> > expected and the metadata requests do not continually get retried.
>> >
>> > In many cases I can understand that creating a KafkaProducer and 
>> > then not using it within 5 minutes is not usual but in our case 
>> > we're creating it when our REST based application starts up and we 
>> > can't guarantee that a message will be published within that time. 
>> > To get around this we are currently posting a test message to the 
>> > topic right after creating the KafkaProducer prevents it happening.
>> >
>> > I'm not sure if this is worthy of a fix but I thought I'd post it 
>> > here in case someone else hits the same problem.
>> >
>> > Regards,
>> > Phil Luckhurst
>> >
>>
>
>

Reply via email to