kafka-jira mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Tamas Mate (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (KAFKA-5138) MirrorMaker doesn't exit on send failure occasionally
Date Thu, 10 Aug 2017 17:39:00 GMT

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

Tamas Mate commented on KAFKA-5138:
-----------------------------------

Hi [~cote],

I don't know how but I could reproduce this only once during the first couple of tries then
I lost it :).
Could you lend me a hand with the reproduction? 

My way of testing was basically blocking the target broker's port with iptables then waiting
for the MM's producer to fail. With a small bruteforce script I tried this ~200 times but
the MirrorMaker stopped with correct error message(TimeoutException)  every time. 

What was you way of reproducing this issue?


> MirrorMaker doesn't exit on send failure occasionally
> -----------------------------------------------------
>
>                 Key: KAFKA-5138
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5138
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.10.2.0
>            Reporter: Dustin Cote
>              Labels: newbie
>
> MirrorMaker with abort.on.send.failure=true does not always exit if the producer closes.
Here is the logic that happens:
> First we encounter a problem producing and force the producer to close
> {code}
> [2017-04-10 07:17:25,137] ERROR Error when sending message to topic mytopicwith key:
20 bytes, value: 314 bytes with error: (org.apache.kafka.clients.producer.internals.ErrorLoggingCallback)
> org.apache.kafka.common.errors.TimeoutException: Expiring 47 record(s) for mytopic-2:
30879 ms has passed since last append
> [2017-04-10 07:17:25,170] INFO Closing producer due to send failure. (kafka.tools.MirrorMaker$)
> [2017-04-10 07:17:25,170] INFO Closing the Kafka producer with timeoutMillis = 0 ms.
(org.apache.kafka.clients.producer.KafkaProducer)
> [2017-04-10 07:17:25,170] INFO Proceeding to force close the producer since pending requests
could not be completed within timeout 0 ms. (org.apache.kafka.clients.producer.KafkaProducer)
> [2017-04-10 07:17:25,170] DEBUG The Kafka producer has closed. (org.apache.kafka.clients.producer.KafkaProducer)
> [2017-04-10 07:17:25,170] ERROR Error when sending message to topic mytopic with key:
20 bytes, value: 313 bytes with error: (org.apache.kafka.clients.producer.internals.ErrorLoggingCallback)
> org.apache.kafka.common.errors.TimeoutException: Expiring 47 record(s) for mytopic-2:
30879 ms has passed since last append
> [2017-04-10 07:17:25,170] INFO Closing producer due to send failure. (kafka.tools.MirrorMaker$)
> [2017-04-10 07:17:25,170] INFO Closing the Kafka producer with timeoutMillis = 0 ms.
(org.apache.kafka.clients.producer.KafkaProducer)
> [2017-04-10 07:17:25,170] INFO Proceeding to force close the producer since pending requests
could not be completed within timeout 0 ms. (org.apache.kafka.clients.producer.KafkaProducer)
> [2017-04-10 07:17:25,170] DEBUG The Kafka producer has closed. (org.apache.kafka.clients.producer.KafkaProducer)
> {code}
> All good there. Then we can't seem to close the producer nicely after about 15 seconds
and so it is forcefully killed:
> {code}
> [2017-04-10 07:17:39,778] ERROR Error when sending message to topic mytopic.subscriptions
with key: 70 bytes, value: null with error: (org.apache.kafka.clients.producer.internals.ErrorLoggingCallback)
> java.lang.IllegalStateException: Producer is closed forcefully.
>         at org.apache.kafka.clients.producer.internals.RecordAccumulator.abortBatches(RecordAccumulator.java:522)
>         at org.apache.kafka.clients.producer.internals.RecordAccumulator.abortIncompleteBatches(RecordAccumulator.java:502)
>         at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:147)
>         at java.lang.Thread.run(Unknown Source)
> [2017-04-10 07:17:39,778] INFO Closing producer due to send failure. (kafka.tools.MirrorMaker$)
> [2017-04-10 07:17:39,778] INFO Closing the Kafka producer with timeoutMillis = 0 ms.
(org.apache.kafka.clients.producer.KafkaProducer)
> [2017-04-10 07:17:39,778] INFO Proceeding to force close the producer since pending requests
could not be completed within timeout 0 ms. (org.apache.kafka.clients.producer.KafkaProducer)
> [2017-04-10 07:17:39,778] DEBUG The Kafka producer has closed. (org.apache.kafka.clients.producer.KafkaProducer)
> [2017-04-10 07:17:39,779] DEBUG Removed sensor with name connections-closed: (org.apache.kafka.common.metrics.Metrics)
> {code}
> After removing some metric sensors for awhile this happens:
> {code}
> [2017-04-10 07:17:39,780] DEBUG Removed sensor with name node-3.latency (org.apache.kafka.common.metrics.Metrics)
> [2017-04-10 07:17:39,780] DEBUG Shutdown of Kafka producer I/O thread has completed.
(org.apache.kafka.clients.producer.internals.Sender)
> [2017-04-10 07:17:41,852] DEBUG Sending Heartbeat request for group mirror-maker-1491619052-teab1-1
to coordinator myhost1:9092 (id: 2147483643 rack: null) (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2017-04-10 07:17:41,953] DEBUG Received successful Heartbeat response for group mirror-maker-1491619052-teab1-1
(org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2017-04-10 07:17:44,875] DEBUG Sending Heartbeat request for group mirror-maker-1491619052-teab1-1
to coordinator myhost1:9092 (id: 2147483643 rack: null) (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> {code}
> This heartbeating goes one for some time until:
> {code}
> [2017-04-10 07:19:57,392] DEBUG Received successful Heartbeat response for group mirror-maker-1491619052-teab1-1
(org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2017-04-10 07:19:57,994] DEBUG Connection with myhost1/123.123.321.321 disconnected
(org.apache.kafka.common.network.Selector)
> java.io.IOException: Connection reset by peer
>         at sun.nio.ch.FileDispatcherImpl.read0(Native Method)
>         at sun.nio.ch.SocketDispatcher.read(Unknown Source)
>         at sun.nio.ch.IOUtil.readIntoNativeBuffer(Unknown Source)
>         at sun.nio.ch.IOUtil.read(Unknown Source)
>         at sun.nio.ch.SocketChannelImpl.read(Unknown Source)
>         at org.apache.kafka.common.network.PlaintextTransportLayer.read(PlaintextTransportLayer.java:110)
>         at org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:81)
>         at org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:71)
>         at org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:169)
>         at org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:150)
>         at org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:355)
>         at org.apache.kafka.common.network.Selector.poll(Selector.java:303)
>         at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:349)
>         at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:226)
>         at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.pollNoWakeup(ConsumerNetworkClient.java:263)
>         at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$HeartbeatThread.run(AbstractCoordinator.java:887)
> [2017-04-10 07:19:57,995] DEBUG Node -1 disconnected. (org.apache.kafka.clients.NetworkClient)
> [2017-04-10 07:19:59,370] DEBUG Updated cluster metadata version 3 to Cluster....
> {code}
> Then we heartbeat again until the process is manually killed:
> {code}
> [2017-04-10 07:22:18,933] DEBUG Received successful Heartbeat response for group mirror-maker-1491619052-teab1-1
(org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2017-04-10 07:22:21,740] DEBUG Sending Heartbeat request for group mirror-maker-1491619052-teab1-1
to coordinator myhost1:9092 (id: 2147483643 rack: null) (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2017-04-10 07:22:21,840] DEBUG Received successful Heartbeat response for group mirror-maker-1491619052-teab1-1
(org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2017-04-10 07:22:24,746] DEBUG Sending Heartbeat request for group mirror-maker-1491619052-teab1-1
to coordinator myhost1:9092 (id: 2147483643 rack: null) (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2017-04-10 07:22:24,847] DEBUG Received successful Heartbeat response for group mirror-maker-1491619052-teab1-1
(org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2017-04-10 07:22:24,847] DEBUG Sending LeaveGroup request to coordinator myhost1:9092
(id: 2147483643 rack: null) for group mirror-maker-1491619052-teab1-1 (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> {code}
> I'm wondering if this is new behavior with the consumer heartbeat thread causing this.




--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message