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

ASF GitHub Bot commented on KAFKA-6541:
---------------------------------------

hachikuji closed pull request #4548: KAFKA-6541: Fixed a StackOverflow bug in 
kafka-coordinator-heartbeat-thread
URL: https://github.com/apache/kafka/pull/4548
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
index 0747e8db146..a0d4ed8f57c 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
@@ -57,6 +57,7 @@
     // flag and the request completion queue below).
     private final Logger log;
     private final KafkaClient client;
+    private boolean disconnecting = false;
     private final UnsentRequests unsent = new UnsentRequests();
     private final Metadata metadata;
     private final Time time;
@@ -387,9 +388,13 @@ private void checkDisconnects(long now) {
     }
 
     public void disconnect(Node node) {
-        failUnsentRequests(node, DisconnectException.INSTANCE);
-
         synchronized (this) {
+            if(disconnecting){
+                return;
+            }
+            disconnecting=true;
+            failUnsentRequests(node, DisconnectException.INSTANCE);            
+
             client.disconnect(node.idString());
         }
 
@@ -483,6 +488,7 @@ public boolean connectionFailed(Node node) {
     public void tryConnect(Node node) {
         synchronized (this) {
             client.ready(node, time.milliseconds());
+            disconnecting = false;
         }
     }
 


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> StackOverflow exceptions in thread 'kafka-coordinator-heartbeat-thread
> ----------------------------------------------------------------------
>
>                 Key: KAFKA-6541
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6541
>             Project: Kafka
>          Issue Type: Bug
>          Components: clients
>    Affects Versions: 1.0.0
>         Environment: Linux
>            Reporter: Anh Le
>            Priority: Major
>
> There's something wrong with our client library when sending heart beats. 
> This bug seems to be identical to this one: 
> [http://mail-archives.apache.org/mod_mbox/kafka-users/201712.mbox/%3CCALte62w6=pJObC+i36BkoqbOLTKsQ=nrddv6dm8abfwb5ps...@mail.gmail.com%3E]
>  
> Here's the log:
>  
> {quote}2018-02-08 13:55:01,102 ERROR 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$HeartbeatThread
>  Uncaught exception in thread 'kafka-coordinator-heartbeat-thread | 
> default-group':
> java.lang.StackOverflowError: null
>  at java.lang.StringBuilder.append(StringBuilder.java:136)
>  at 
> org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:302)
>  at 
> org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:271)
>  at org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:233)
>  at org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:173)
>  at 
> ch.qos.logback.classic.spi.LoggingEvent.getFormattedMessage(LoggingEvent.java:293)
>  at 
> ch.qos.logback.classic.spi.LoggingEvent.prepareForDeferredProcessing(LoggingEvent.java:206)
>  at 
> ch.qos.logback.core.OutputStreamAppender.subAppend(OutputStreamAppender.java:223)
>  at 
> ch.qos.logback.core.OutputStreamAppender.append(OutputStreamAppender.java:102)
>  at 
> ch.qos.logback.core.UnsynchronizedAppenderBase.doAppend(UnsynchronizedAppenderBase.java:84)
>  at 
> ch.qos.logback.core.spi.AppenderAttachableImpl.appendLoopOnAppenders(AppenderAttachableImpl.java:51)
>  at ch.qos.logback.classic.Logger.appendLoopOnAppenders(Logger.java:270)
>  at ch.qos.logback.classic.Logger.callAppenders(Logger.java:257)
>  at ch.qos.logback.classic.Logger.buildLoggingEventAndAppend(Logger.java:421)
>  at ch.qos.logback.classic.Logger.filterAndLog_1(Logger.java:398)
>  at ch.qos.logback.classic.Logger.info(Logger.java:583)
>  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)
>  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)
>  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)
>  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)
> {quote}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to