Rajini Sivaram created KAFKA-7280:
-------------------------------------

             Summary: ConcurrentModificationException in FetchSessionHandler in 
heartbeat thread
                 Key: KAFKA-7280
                 URL: https://issues.apache.org/jira/browse/KAFKA-7280
             Project: Kafka
          Issue Type: Bug
          Components: consumer
    Affects Versions: 2.0.0, 1.1.1
            Reporter: Rajini Sivaram
            Assignee: Rajini Sivaram
             Fix For: 1.1.2, 2.0.1, 2.1.0


Request/response handling in FetchSessionHandler is not thread-safe. But we are 
using it in Kafka consumer without any synchronization even though poll() from 
heartbeat thread can process responses. Heartbeat thread holds the coordinator 
lock while processing its poll and responses, making other operations involving 
the group coordinator safe. We also need to lock FetchSessionHandler for the 
operations that update or read FetchSessionHandler#sessionPartitions.

This exception is from a system test run on trunk of 
TestSecurityRollingUpgrade.test_rolling_upgrade_sasl_mechanism_phase_two:
{quote}
[2018-08-12 06:13:22,316] ERROR [Consumer clientId=console-consumer, 
groupId=group] Heartbeat thread failed due to unexpected error 
(org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
java.util.ConcurrentModificationException
        at 
java.util.LinkedHashMap$LinkedHashIterator.nextNode(LinkedHashMap.java:719)
        at 
java.util.LinkedHashMap$LinkedKeyIterator.next(LinkedHashMap.java:742)
        at 
org.apache.kafka.clients.FetchSessionHandler.responseDataToLogString(FetchSessionHandler.java:362)
        at 
org.apache.kafka.clients.FetchSessionHandler.handleResponse(FetchSessionHandler.java:424)
        at 
org.apache.kafka.clients.consumer.internals.Fetcher$1.onSuccess(Fetcher.java:216)
        at 
org.apache.kafka.clients.consumer.internals.Fetcher$1.onSuccess(Fetcher.java:206)
        at 
org.apache.kafka.clients.consumer.internals.RequestFuture.fireSuccess(RequestFuture.java:167)
        at 
org.apache.kafka.clients.consumer.internals.RequestFuture.complete(RequestFuture.java:127)
        at 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.fireCompletion(ConsumerNetworkClient.java:575)
        at 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.firePendingCompletedRequests(ConsumerNetworkClient.java:389)
        at 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:297)
        at 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.pollNoWakeup(ConsumerNetworkClient.java:304)
        at 
org.apache.kafka.clients.consumer.internals.AbstractCoordinator$HeartbeatThread.run(AbstractCoordinator.java:996)

{quote}
 

 



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

Reply via email to