[jira] [Commented] (KAFKA-5138) MirrorMaker doesn't exit on send failure occasionally

2017-08-17 Thread Tamas Mate (JIRA)

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

Tamas Mate commented on KAFKA-5138:
---

Hey Guys,

I think I have found something.
My guess is that when the AbstactCoordinator.maybeLeaveGroup(...) function's 
pollNoWakeUp part throws an exception, then it can not call resetGeneration() 
and the HearBeatThread stays in STABLE state. The broker won't be notified 
about the consumer's leave request so it thinks that everything is all right 
and responds to its requests.

If this is the case it seems impossible to fix it at consumer side, maybe with 
a new config parameter (leave retry?).
The other option could be a MirrorMaker fix for example when the producer dies 
shoot the consumers in the head.

[~hachikuji] What do you think about these? Does it worth to continue the 
investigation this way?

> 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)
> [20

[jira] [Commented] (KAFKA-5138) MirrorMaker doesn't exit on send failure occasionally

2017-08-11 Thread Tamas Mate (JIRA)

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

Tamas Mate commented on KAFKA-5138:
---

[~cote] Cool, thanks for the tip, I have just installed Jmeter, then I will 
keep going this way. :)

> 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 f

[jira] [Commented] (KAFKA-5138) MirrorMaker doesn't exit on send failure occasionally

2017-08-11 Thread Dustin Cote (JIRA)

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

Dustin Cote commented on KAFKA-5138:


[~tamas.mate] I've never managed to reproduce it. It came out of a poorly 
performing destination cluster, so instead of iptables, you could try running 
with an extremely low `request.timeout.ms` on the mirrormaker producer?

> 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 
> 

[jira] [Commented] (KAFKA-5138) MirrorMaker doesn't exit on send failure occasionally

2017-08-10 Thread Tamas Mate (JIRA)

[ 
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-14916190

[jira] [Commented] (KAFKA-5138) MirrorMaker doesn't exit on send failure occasionally

2017-08-07 Thread Dustin Cote (JIRA)

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

Dustin Cote commented on KAFKA-5138:


[~tamas.mate] I have no plans to work on this at this time. It's all yours :)

> 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-ma

[jira] [Commented] (KAFKA-5138) MirrorMaker doesn't exit on send failure occasionally

2017-08-07 Thread Tamas Mate (JIRA)

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

Tamas Mate commented on KAFKA-5138:
---

Hi [~cote],
have you planned to work on this? Because I would like to check this out.

> 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 
> mir