[
https://issues.apache.org/jira/browse/KAFKA-1257?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898611#comment-13898611
]
Jay Kreps commented on KAFKA-1257:
----------------------------------
Hey Neha, I'm not 100% sure the cause of this issue. This log doesn't show it,
but the other log seemed to indicate that we first got an error from the
Selector for trying to initiate a second concurrent send. If that is the case I
suspect it made it into our in-flight request list first and from then on we
are trying to correlate responses with the wrong request.
Irrespective the metadata request issue is definitely real. This patch address
this by only initiating a metadata request to a node that doesn't have
in-flight requests.
I also reversed the preference so we first initiate metadata request and then
select partitions to produce to. This ensures that we prefer metadata requests
over produce requests (otherwise a busy producer could be starved out
indefinitely from sending its metadata requests).
> New producer runs into out of order producer responses
> ------------------------------------------------------
>
> Key: KAFKA-1257
> URL: https://issues.apache.org/jira/browse/KAFKA-1257
> Project: Kafka
> Issue Type: Sub-task
> Components: producer
> Affects Versions: 0.9.0
> Reporter: Neha Narkhede
> Priority: Blocker
> Attachments: KAFKA-1257.patch, KAFKA-1257_2014-02-11_17:11:54.patch,
> KAFKA-1257_2014-02-11_17:14:48.patch
>
>
> Ran into the following bug while running the new mirror maker on one large
> topic -
> java.lang.IllegalStateException: Correlation id for response (13502150) does
> not match request (13502149)
> at
> org.apache.kafka.clients.producer.internals.Sender.correlate(Sender.java:333)
> at
> org.apache.kafka.clients.producer.internals.Sender.handleResponses(Sender.java:289)
> at
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:158)
> at
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:98)
> at java.lang.Thread.run(Thread.java:619)
> java.lang.IllegalStateException: Correlation id for response (13502151) does
> not match request (13502150)
> at
> org.apache.kafka.clients.producer.internals.Sender.correlate(Sender.java:333)
> at
> org.apache.kafka.clients.producer.internals.Sender.handleResponses(Sender.java:289)
> at
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:158)
> at
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:98)
> at java.lang.Thread.run(Thread.java:619)
> java.lang.IllegalStateException: Correlation id for response (13502152) does
> not match request (13502151)
> at
> org.apache.kafka.clients.producer.internals.Sender.correlate(Sender.java:333)
> at
> org.apache.kafka.clients.producer.internals.Sender.handleResponses(Sender.java:289)
> at
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:158)
> at
> org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:98)
--
This message was sent by Atlassian JIRA
(v6.1.5#6160)