Re: [KAFKA-5138] MirrorMaker doesn't exit on send failure occasionally

2017-08-15 Thread Tamás Máté
Hey,

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.

What do you think about these?

Although, I still couldn't repro the issue, will try to do that tomorrow. :)

Best regards,
Tamas

On 15 August 2017 at 14:49, Tamás Máté  wrote:

> Hi Guys,
>
> I have just started to work on this ticket a little more than a week ago:
> https://issues.apache.org/jira/browse/KAFKA-5138
>
> I could not reproduce it sadly, but from the logs Dustin gave and from the
> code it seems like this might not be just a MirrorMaker issue but a
> consumer one.
>
> My theory is
>  1) MM send failure happens because of heavy load
>  2) MM starts to close its producer
>  3) during MM shutdown and the source server starts a consumer rebalance
> (the consumers couldn't respond because of the heavy load)
>  4) heartbeat response gets delayed
>  5) MM producer closed, but MM gets a heartbeat response and resets the
> connection
>  6) because there is thread left in the JVM it can't shut down
>  7) MM hangs
>
> Maybe the order is a bit different, I couldn't prove it without
> reproduction.
>
> I played with the following configs under 100ms and then stress tested the
> source cluster with JMeter.
>  - request.timeout.ms
>  - replica.lag.time.max.ms
>  - session.timeout.ms
>  - group.min.session.timeout.ms
>  - group.max.session.timeout.ms
>  - heartbeat.interval.ms
>
> Could you give me some pointers how could I reproduce this issue?
>
> Thanks,
> Tamas
>
>


[KAFKA-5138] MirrorMaker doesn't exit on send failure occasionally

2017-08-15 Thread Tamás Máté
Hi Guys,

I have just started to work on this ticket a little more than a week ago:
https://issues.apache.org/jira/browse/KAFKA-5138

I could not reproduce it sadly, but from the logs Dustin gave and from the
code it seems like this might not be just a MirrorMaker issue but a
consumer one.

My theory is
 1) MM send failure happens because of heavy load
 2) MM starts to close its producer
 3) during MM shutdown and the source server starts a consumer rebalance
(the consumers couldn't respond because of the heavy load)
 4) heartbeat response gets delayed
 5) MM producer closed, but MM gets a heartbeat response and resets the
connection
 6) because there is thread left in the JVM it can't shut down
 7) MM hangs

Maybe the order is a bit different, I couldn't prove it without
reproduction.

I played with the following configs under 100ms and then stress tested the
source cluster with JMeter.
 - request.timeout.ms
 - replica.lag.time.max.ms
 - session.timeout.ms
 - group.min.session.timeout.ms
 - group.max.session.timeout.ms
 - heartbeat.interval.ms

Could you give me some pointers how could I reproduce this issue?

Thanks,
Tamas


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

2017-04-27 Thread Ismael Juma (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-5138?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma updated KAFKA-5138:
---
Labels: newbie  (was: )

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

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

2017-04-27 Thread Dustin Cote (JIRA)
Dustin Cote created KAFKA-5138:
--

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


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