You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Steven Zhen Wu (JIRA)" <ji...@apache.org> on 2015/02/04 18:15:36 UTC

[jira] [Commented] (KAFKA-1919) Metadata request issued with no backoff in new producer if there are no topics

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

Steven Zhen Wu commented on KAFKA-1919:
---------------------------------------

enabled TRACE level logging. it shows tight loop with metadata request. two metadata requests in the same milli-seconds

kafka-producer-network-thread | foo 08:58:34,917 TRACE NetworkClient:315 - Ignoring empty metadata response with correlation id 1473532.
kafka-producer-network-thread | foo 08:58:34,917 DEBUG NetworkClient:385 - Trying to send metadata request to node -2
kafka-producer-network-thread | foo 08:58:34,917 DEBUG NetworkClient:390 - Sending metadata request ClientRequest(expectResponse=true, payload=null, request=RequestSend(header={api_key=3,api_version=0,correlation_id=1473533,client_id=foo}, body={topics=[]})) to node -2
kafka-producer-network-thread | foo 08:58:34,917 TRACE NetworkClient:315 - Ignoring empty metadata response with correlation id 1473533.
kafka-producer-network-thread | foo 08:58:34,917 DEBUG NetworkClient:385 - Trying to send metadata request to node -2
kafka-producer-network-thread | foo 08:58:34,918 DEBUG NetworkClient:390 - Sending metadata request ClientRequest(expectResponse=true, payload=null, request=RequestSend(header={api_key=3,api_version=0,correlation_id=1473534,client_id=foo}, body={topics=[]})) to node -2
kafka-producer-network-thread | foo 08:58:34,918 TRACE NetworkClient:315 - Ignoring empty metadata response with correlation id 1473534.
kafka-producer-network-thread | foo 08:58:34,918 DEBUG NetworkClient:385 - Trying to send metadata request to node -2
kafka-producer-network-thread | foo 08:58:34,918 DEBUG NetworkClient:390 - Sending metadata request ClientRequest(expectResponse=true, payload=null, request=RequestSend(header={api_key=3,api_version=0,correlation_id=1473535,client_id=foo}, body={topics=[]})) to node -2
kafka-producer-network-thread | foo 08:58:34,918 TRACE NetworkClient:315 - Ignoring empty metadata response with correlation id 1473535.
kafka-producer-network-thread | foo 08:58:34,918 DEBUG NetworkClient:385 - Trying to send metadata request to node -2

> Metadata request issued with no backoff in new producer if there are no topics
> ------------------------------------------------------------------------------
>
>                 Key: KAFKA-1919
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1919
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.8.2
>            Reporter: Jay Kreps
>
> Original report:
> We have observed high cpu and high network traffic problem when
> 1) cluster (0.8.1.1) has no topic
> 2) KafkaProducer (0.8.2-beta) object is created without sending any traffic
> We have observed such problem twice. In both cases, problem went away
> immediately after one/any topic is created.
> Is this a known issue? Just want to check with the community first before I
> spend much time to reproduce it.
> I couldn't reproduce the issue with similar setup with unit test code in
> IDE. start two brokers with no topic locally on my laptop. create a
> KafkaProducer object without sending any msgs. but I only tested with
> 0.8.2-beta for both broker and producer.
> Issue exists in 0.8.2 as well:
> I have re-run my unit test with 0.8.2.0. same tight-loop problem happened
> after a few mins.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)