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

PoAn Yang commented on KAFKA-17223:
-----------------------------------

I create a draft PR to avoid modifying `runnable.pendingCalls.add(this);` to 
`runnable.newCalls.add(this);`. The step is like following:

[https://github.com/apache/kafka/pull/16753]
 # Use for-loop instead of iterator to check `pendingCalls`.
 # Use a list `toRemove` to collect removed calls in `pendingCalls`.
 # Use `pendingCalls.remove(call)` instead of 
`pendingCalls.removeAll(toRemove)` to avoid removing all matched elements.

> Retrying the call after encoutering UnsupportedVersionException will cause 
> ConcurrentModificationException
> ----------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-17223
>                 URL: https://issues.apache.org/jira/browse/KAFKA-17223
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Chia-Ping Tsai
>            Assignee: PoAn Yang
>            Priority: Minor
>
> {code:java}
> [2024-07-31 07:11:03,928] ERROR Uncaught exception in thread 
> 'kafka-admin-client-thread | adminclient-1': 
> (org.apache.kafka.common.utils.KafkaThread:51)
> java.util.ConcurrentModificationException
>       at 
> java.base/java.util.ArrayList$Itr.checkForComodification(ArrayList.java:1013)
>       at java.base/java.util.ArrayList$Itr.remove(ArrayList.java:981)
>       at 
> org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.maybeDrainPendingCalls(KafkaAdminClient.java:1207)
>       at 
> org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.processRequests(KafkaAdminClient.java:1510)
>       at 
> org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.run(KafkaAdminClient.java:1464)
>       at java.base/java.lang.Thread.run(Thread.java:840)
> {code}
> The steps producing above error are shown below.
> 1. maybeDrainPendingCall[0] encounter error when calling 
> `call.nodeProvider.provide();`[1]
> 2. `runnable.pendingCalls.add(this)`[2] adds the call back to `pendingCalls`
> 3. `pendingIter.remove();` tries to remove item from the modified array list.
> IMHO, we should add call back to `newCalls` rather than `pendingCalls`.  This 
> approach is to revert a part of KAFKA-12432
> [0] 
> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java#L1206
> [1] 
> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java#L1219
> [2] 
> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java#L927
> [3] 
> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java#L1219



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to