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

Joerg Heinicke commented on KAFKA-6366:
---------------------------------------

Sorry for the delay. Even though I planned to at least try to extract the 
information from the system over Christmas holidays I haven't managed. Now back 
from vacation I have done so. I attached the log file 
[^ConverterProcessor_DEBUG.zip] (which is around 700k lines and 150 MB for 
about 5 mins!). I don't get additional hints on the issue, not sure whether it 
helps you to confirm the scenario.

We don't have particular test scenarios to test the patch which means we would 
have to run this directly in production - which I'm not too comfortable with if 
you could not even confirm the scenario yet. Another question which comes to my 
mind is how the consumer will behave in case we hit the scenario with the patch 
applied since apparently all other threads are still able to commit while the 
failing thread (pool-5-thread-5 in the attached log file) marked the 
coordinator dead, i.e. what is the expected and probably originally intended 
behavior. And on the most basic and practical side: How do I get a Kafka 
distribution with the patch applied? Apparently I will have to build it myself. 
Can you give me some kick-off hints? Is the documentation at 
https://github.com/apache/kafka all I need?

> StackOverflowError in kafka-coordinator-heartbeat-thread
> --------------------------------------------------------
>
>                 Key: KAFKA-6366
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6366
>             Project: Kafka
>          Issue Type: Bug
>          Components: consumer
>    Affects Versions: 1.0.0
>            Reporter: Joerg Heinicke
>            Assignee: Jason Gustafson
>         Attachments: 6366.v1.txt, ConverterProcessor.zip, 
> ConverterProcessor_DEBUG.zip, Screenshot-2017-12-19 21.35-22.10 processing.png
>
>
> With Kafka 1.0 our consumer groups fall into a permanent cycle of rebalancing 
> once a StackOverflowError in the heartbeat thread occurred due to 
> connectivity issues of the consumers to the coordinating broker:
> Immediately before the exception there are hundreds, if not thousands of log 
> entries of following type:
> 2017-12-12 16:23:12.361 [kafka-coordinator-heartbeat-thread | 
> my-consumer-group] INFO  - [Consumer clientId=consumer-4, 
> groupId=my-consumer-group] Marking the coordinator <IP>:<Port> (id: 
> 2147483645 rack: null) dead
> The exceptions always happen somewhere in the DateFormat code, even 
> though at different lines.
> 2017-12-12 16:23:12.363 [kafka-coordinator-heartbeat-thread | 
> my-consumer-group] ERROR - Uncaught exception in thread 
> 'kafka-coordinator-heartbeat-thread | my-consumer-group':
> java.lang.StackOverflowError
>          at 
> java.text.DateFormatSymbols.getProviderInstance(DateFormatSymbols.java:362)
>          at 
> java.text.DateFormatSymbols.getInstance(DateFormatSymbols.java:340)
>          at java.util.Calendar.getDisplayName(Calendar.java:2110)
>          at java.text.SimpleDateFormat.subFormat(SimpleDateFormat.java:1125)
>          at java.text.SimpleDateFormat.format(SimpleDateFormat.java:966)
>          at java.text.SimpleDateFormat.format(SimpleDateFormat.java:936)
>          at java.text.DateFormat.format(DateFormat.java:345)
>          at 
> org.apache.log4j.helpers.PatternParser$DatePatternConverter.convert(PatternParser.java:443)
>          at 
> org.apache.log4j.helpers.PatternConverter.format(PatternConverter.java:65)
>          at org.apache.log4j.PatternLayout.format(PatternLayout.java:506)
>          at 
> org.apache.log4j.WriterAppender.subAppend(WriterAppender.java:310)
>          at org.apache.log4j.WriterAppender.append(WriterAppender.java:162)
>          at 
> org.apache.log4j.AppenderSkeleton.doAppend(AppenderSkeleton.java:251)
>          at 
> org.apache.log4j.helpers.AppenderAttachableImpl.appendLoopOnAppenders(AppenderAttachableImpl.java:66)
>          at org.apache.log4j.Category.callAppenders(Category.java:206)
>          at org.apache.log4j.Category.forcedLog(Category.java:391)
>          at org.apache.log4j.Category.log(Category.java:856)
>          at 
> org.slf4j.impl.Log4jLoggerAdapter.info(Log4jLoggerAdapter.java:324)
>          at 
> org.apache.kafka.common.utils.LogContext$KafkaLogger.info(LogContext.java:341)
>          at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.coordinatorDead(AbstractCoordinator.java:649)
>          at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onFailure(AbstractCoordinator.java:797)
>          at 
> org.apache.kafka.clients.consumer.internals.RequestFuture$1.onFailure(RequestFuture.java:209)
>          at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.fireFailure(RequestFuture.java:177)
>          at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.raise(RequestFuture.java:147)
>          at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.fireCompletion(ConsumerNetworkClient.java:496)
> ...
> the following 9 lines are repeated around hundred times.
> ...
>          at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.fireCompletion(ConsumerNetworkClient.java:496)
>          at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.firePendingCompletedRequests(ConsumerNetworkClient.java:353)
>          at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.failUnsentRequests(ConsumerNetworkClient.java:416)
>          at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.disconnect(ConsumerNetworkClient.java:388)
>          at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.coordinatorDead(AbstractCoordinator.java:653)
>          at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onFailure(AbstractCoordinator.java:797)
>          at 
> org.apache.kafka.clients.consumer.internals.RequestFuture$1.onFailure(RequestFuture.java:209)
>          at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.fireFailure(RequestFuture.java:177)
>          at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.raise(RequestFuture.java:147)
>          at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.fireCompletion(ConsumerNetworkClient.java:496)



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

Reply via email to