You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/05/15 16:05:37 UTC

[GitHub] [kafka] lbradstreet opened a new pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

lbradstreet opened a new pull request #10704:
URL: https://github.com/apache/kafka/pull/10704


   Recently we have noticed multiple instances where KafkaProducers have failed to constructer due to the following exception:
   
   ```
   org.apache.kafka.common.KafkaException: Failed to construct kafka producer at org.apache.kafka.clients.producer.KafkaProducer.<init>(KafkaProducer.java:440) at org.apache.kafka.clients.producer.KafkaProducer.<init>(KafkaProducer.java:291) at org.apache.kafka.clients.producer.KafkaProducer.<init>(KafkaProducer.java:318) java.base/java.lang.Thread.run(Thread.java:832) Caused by: java.util.ConcurrentModificationException at java.base/java.util.HashMap$HashIterator.nextNode(HashMap.java:1584) at java.base/java.util.HashMap$KeyIterator.next(HashMap.java:1607) at java.base/java.util.AbstractSet.removeAll(AbstractSet.java:171) at org.apache.kafka.common.config.AbstractConfig.unused(AbstractConfig.java:221) at org.apache.kafka.common.config.AbstractConfig.logUnused(AbstractConfig.java:379) at org.apache.kafka.clients.producer.KafkaProducer.<init>(KafkaProducer.java:433) ... 9 more exception.class:org.apache.kafka.common.KafkaException exception.message:Failed to construct kafka producer
   ```
   
   This is due to the fact that `used` below is a synchronized set. `used` is being modified while removeAll is being called. This may be due to the way that keys are added to it when used. Switching to a CopyOnWriteArraySet avoids this issue as it support concurrent iteration.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#issuecomment-841764123


   I was thinking about this and it's a bit weird how this happens. When we call `logUnused`, there should be nothing else calling `used`. Any ideas how this can happen?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#discussion_r640788402



##########
File path: clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java
##########
@@ -508,6 +509,55 @@ public void testDocumentationOfExpectNull() {
         assertNull(config.documentationOf("xyz"));
     }
 
+    @Test
+    public void testConcurrentUnusedUse() throws InterruptedException {

Review comment:
       I think I'd remove it or at least tag it as an integration test.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] lbradstreet commented on a change in pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
lbradstreet commented on a change in pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#discussion_r633028316



##########
File path: clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
##########
@@ -106,7 +107,7 @@ public AbstractConfig(ConfigDef definition, Map<?, ?> originals,  Map<String, ?>
 
         this.originals = resolveConfigVariables(configProviderProps, (Map<String, Object>) originals);
         this.values = definition.parse(this.originals);
-        this.used = Collections.synchronizedSet(new HashSet<>());
+        this.used = new CopyOnWriteArraySet<>();

Review comment:
       Yeah, I normally don't like them very much either but I thought the number of operations on it would be small and it'd be OK. Given that this used map is updated on every use, maybe that assumption could end up being wrong in some use of AbstractConfig. I'll switch it to ConcurrentHashSet.

##########
File path: clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
##########
@@ -106,7 +107,7 @@ public AbstractConfig(ConfigDef definition, Map<?, ?> originals,  Map<String, ?>
 
         this.originals = resolveConfigVariables(configProviderProps, (Map<String, Object>) originals);
         this.values = definition.parse(this.originals);
-        this.used = Collections.synchronizedSet(new HashSet<>());
+        this.used = new CopyOnWriteArraySet<>();

Review comment:
       Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] lbradstreet edited a comment on pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
lbradstreet edited a comment on pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#issuecomment-849144204


   @ijuma I figured out the cause with some state change checks. The problem is that we're passing a recording map through to the channel/principal builders, and the recording map updates ignore. 
   ```
   	at org.apache.kafka.clients.producer.ProducerConfig.ignore(ProducerConfig.java:569)
   	at org.apache.kafka.common.config.AbstractConfig$RecordingMap.get(AbstractConfig.java:638)
   	at org.apache.kafka.common.network.ChannelBuilders.createPrincipalBuilder(ChannelBuilders.java:242)
   	at org.apache.kafka.common.network.PlaintextChannelBuilder$PlaintextAuthenticator.<init>(PlaintextChannelBuilder.java:96)
   	at org.apache.kafka.common.network.PlaintextChannelBuilder$PlaintextAuthenticator.<init>(PlaintextChannelBuilder.java:89)
   	at org.apache.kafka.common.network.PlaintextChannelBuilder.lambda$buildChannel$0(PlaintextChannelBuilder.java:66)
   	at org.apache.kafka.common.network.KafkaChannel.<init>(KafkaChannel.java:174)
   	at org.apache.kafka.common.network.KafkaChannel.<init>(KafkaChannel.java:164)
   	at org.apache.kafka.common.network.PlaintextChannelBuilder.buildChannel(PlaintextChannelBuilder.java:79)
   	at org.apache.kafka.common.network.PlaintextChannelBuilder.buildChannel(PlaintextChannelBuilder.java:67)
   	at org.apache.kafka.common.network.Selector.buildAndAttachKafkaChannel(Selector.java:356)
   	at org.apache.kafka.common.network.Selector.registerChannel(Selector.java:347)
   	at org.apache.kafka.common.network.Selector.connect(Selector.java:274)
   	at org.apache.kafka.clients.NetworkClient.initiateConnect(NetworkClient.java:1097)
   	at org.apache.kafka.clients.NetworkClient.access$700(NetworkClient.java:87)
   	at org.apache.kafka.clients.NetworkClient$DefaultMetadataUpdater.maybeUpdate(NetworkClient.java:1276)
   	at org.apache.kafka.clients.NetworkClient$DefaultMetadataUpdater.maybeUpdate(NetworkClient.java:1164)
   	at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:637)
   	at org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:327)
   	at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:242)
   ```
   
   If we're going to use these recording maps as we use them today I think this PR is a reasonable fix.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#discussion_r633019749



##########
File path: clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
##########
@@ -106,7 +107,7 @@ public AbstractConfig(ConfigDef definition, Map<?, ?> originals,  Map<String, ?>
 
         this.originals = resolveConfigVariables(configProviderProps, (Map<String, Object>) originals);
         this.values = definition.parse(this.originals);
-        this.used = Collections.synchronizedSet(new HashSet<>());
+        this.used = new CopyOnWriteArraySet<>();

Review comment:
       How about a ConcurrentHashSet instead? Not a big fan of CopyOnWriteArraySet.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] lbradstreet commented on pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
lbradstreet commented on pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#issuecomment-841831977


   @ijuma I'm admittedly very confused by this too. I followed the producer configs through everywhere it's passed and can't see anywhere that it seems likely to be called concurrently. There are a few places that seem suspicious (channel builder code, etc) by the time it calls logUnused.
   
   `doSend` will use it later on, but by then the constructor will have had to have returned:
   ```
               try {
                   serializedKey = keySerializer.serialize(record.topic(), record.headers(), record.key());
               } catch (ClassCastException cce) {
                   throw new SerializationException("Can't convert key of class " + record.key().getClass().getName() +
                           " to class " + producerConfig.getClass(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG).getName() +
                           " specified in key.serializer", cce);
               }
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#discussion_r640186978



##########
File path: clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
##########
@@ -106,7 +107,7 @@ public AbstractConfig(ConfigDef definition, Map<?, ?> originals,  Map<String, ?>
 
         this.originals = resolveConfigVariables(configProviderProps, (Map<String, Object>) originals);
         this.values = definition.parse(this.originals);
-        this.used = Collections.synchronizedSet(new HashSet<>());
+        this.used = ConcurrentHashMap.newKeySet();

Review comment:
       Can we initialize this where the field is defined and add a comment on why it has to be concurrent?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] lbradstreet commented on a change in pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
lbradstreet commented on a change in pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#discussion_r632977199



##########
File path: clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java
##########
@@ -508,6 +509,55 @@ public void testDocumentationOfExpectNull() {
         assertNull(config.documentationOf("xyz"));
     }
 
+    @Test
+    public void testConcurrentUnusedUse() throws InterruptedException {

Review comment:
       I'm not sure this test adds much value from here but it picks up the issue and takes about 1.5s to run so I am happy to leave it in or remove it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] lbradstreet commented on a change in pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
lbradstreet commented on a change in pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#discussion_r633028316



##########
File path: clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
##########
@@ -106,7 +107,7 @@ public AbstractConfig(ConfigDef definition, Map<?, ?> originals,  Map<String, ?>
 
         this.originals = resolveConfigVariables(configProviderProps, (Map<String, Object>) originals);
         this.values = definition.parse(this.originals);
-        this.used = Collections.synchronizedSet(new HashSet<>());
+        this.used = new CopyOnWriteArraySet<>();

Review comment:
       Yeah, I normally don't like them very much either but I thought the number of operations on it would be small and it'd be OK. Given that this used map is updated on every use, maybe that assumption could end up being wrong in some use of AbstractConfig. I'll switch it to ConcurrentHashSet.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#discussion_r633019749



##########
File path: clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
##########
@@ -106,7 +107,7 @@ public AbstractConfig(ConfigDef definition, Map<?, ?> originals,  Map<String, ?>
 
         this.originals = resolveConfigVariables(configProviderProps, (Map<String, Object>) originals);
         this.values = definition.parse(this.originals);
-        this.used = Collections.synchronizedSet(new HashSet<>());
+        this.used = new CopyOnWriteArraySet<>();

Review comment:
       How about a ConcurrentHashSet instead? Not a big fan of CopyOnWriteArraySet.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] lbradstreet commented on a change in pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
lbradstreet commented on a change in pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#discussion_r640820748



##########
File path: clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java
##########
@@ -508,6 +509,55 @@ public void testDocumentationOfExpectNull() {
         assertNull(config.documentationOf("xyz"));
     }
 
+    @Test
+    public void testConcurrentUnusedUse() throws InterruptedException {

Review comment:
       I don't think it adds much value so I'd rather delete it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#issuecomment-849934626


   Unrelated flaky tests:
   
   > Build / JDK 15 and Scala 2.13 / kafka.network.ConnectionQuotasTest.testListenerConnectionRateLimitWhenActualRateAboveLimit()
   > Build / JDK 11 and Scala 2.13 / kafka.server.RaftClusterTest.testCreateClusterAndCreateAndManyTopicsWithManyPartitions()
   > Build / JDK 11 and Scala 2.13 / kafka.server.RaftClusterTest.testCreateClusterAndCreateAndManyTopics()
   > Build / JDK 8 and Scala 2.12 / kafka.api.PlaintextConsumerTest.testMultiConsumerDefaultAssignment()
   > Build / JDK 8 and Scala 2.12 / org.apache.kafka.streams.integration.KTableKTableForeignKeyInnerJoinMultiIntegrationTest.shouldInnerJoinMultiPartitionQueryable


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] lbradstreet commented on a change in pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
lbradstreet commented on a change in pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#discussion_r640706594



##########
File path: clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
##########
@@ -106,7 +107,7 @@ public AbstractConfig(ConfigDef definition, Map<?, ?> originals,  Map<String, ?>
 
         this.originals = resolveConfigVariables(configProviderProps, (Map<String, Object>) originals);
         this.values = definition.parse(this.originals);
-        this.used = Collections.synchronizedSet(new HashSet<>());
+        this.used = ConcurrentHashMap.newKeySet();

Review comment:
       Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#issuecomment-841764123


   I was thinking about this and it's a bit weird how this happens. When we call `logUnused`, there should be nothing else calling `used`. Any ideas how this can happen?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] lbradstreet commented on pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
lbradstreet commented on pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#issuecomment-849144204


   @ijuma I figured out the cause with some state change checks. The problem is that we're passing a recording map through to the channel/principal builders, and the recording map updates ignore. 
   ```
   	at org.apache.kafka.clients.producer.ProducerConfig.ignore(ProducerConfig.java:569)
   	at org.apache.kafka.common.config.AbstractConfig$RecordingMap.get(AbstractConfig.java:638)
   	at org.apache.kafka.common.network.ChannelBuilders.createPrincipalBuilder(ChannelBuilders.java:242)
   	at org.apache.kafka.common.network.PlaintextChannelBuilder$PlaintextAuthenticator.<init>(PlaintextChannelBuilder.java:96)
   	at org.apache.kafka.common.network.PlaintextChannelBuilder$PlaintextAuthenticator.<init>(PlaintextChannelBuilder.java:89)
   	at org.apache.kafka.common.network.PlaintextChannelBuilder.lambda$buildChannel$0(PlaintextChannelBuilder.java:66)
   	at org.apache.kafka.common.network.KafkaChannel.<init>(KafkaChannel.java:174)
   	at org.apache.kafka.common.network.KafkaChannel.<init>(KafkaChannel.java:164)
   	at org.apache.kafka.common.network.PlaintextChannelBuilder.buildChannel(PlaintextChannelBuilder.java:79)
   	at org.apache.kafka.common.network.PlaintextChannelBuilder.buildChannel(PlaintextChannelBuilder.java:67)
   	at org.apache.kafka.common.network.Selector.buildAndAttachKafkaChannel(Selector.java:356)
   	at org.apache.kafka.common.network.Selector.registerChannel(Selector.java:347)
   	at org.apache.kafka.common.network.Selector.connect(Selector.java:274)
   	at org.apache.kafka.clients.NetworkClient.initiateConnect(NetworkClient.java:1097)
   	at org.apache.kafka.clients.NetworkClient.access$700(NetworkClient.java:87)
   	at org.apache.kafka.clients.NetworkClient$DefaultMetadataUpdater.maybeUpdate(NetworkClient.java:1276)
   	at org.apache.kafka.clients.NetworkClient$DefaultMetadataUpdater.maybeUpdate(NetworkClient.java:1164)
   	at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:637)
   	at org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:327)
   	at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:242)
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] lbradstreet commented on a change in pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
lbradstreet commented on a change in pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#discussion_r640823494



##########
File path: clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java
##########
@@ -508,6 +509,55 @@ public void testDocumentationOfExpectNull() {
         assertNull(config.documentationOf("xyz"));
     }
 
+    @Test
+    public void testConcurrentUnusedUse() throws InterruptedException {

Review comment:
       Done.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] lbradstreet commented on pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
lbradstreet commented on pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#issuecomment-841831977


   @ijuma I'm admittedly very confused by this too. I followed the producer configs through everywhere it's passed and can't see anywhere that it seems likely to be called concurrently. There are a few places that seem suspicious (channel builder code, etc) by the time it calls logUnused.
   
   `doSend` will use it later on, but by then the constructor will have had to have returned:
   ```
               try {
                   serializedKey = keySerializer.serialize(record.topic(), record.headers(), record.key());
               } catch (ClassCastException cce) {
                   throw new SerializationException("Can't convert key of class " + record.key().getClass().getName() +
                           " to class " + producerConfig.getClass(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG).getName() +
                           " specified in key.serializer", cce);
               }
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma merged pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
ijuma merged pull request #10704:
URL: https://github.com/apache/kafka/pull/10704


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] lbradstreet commented on a change in pull request #10704: KAFKA-12791: ConcurrentModificationException in AbstractConfig use by KafkaProducer

Posted by GitBox <gi...@apache.org>.
lbradstreet commented on a change in pull request #10704:
URL: https://github.com/apache/kafka/pull/10704#discussion_r633029957



##########
File path: clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
##########
@@ -106,7 +107,7 @@ public AbstractConfig(ConfigDef definition, Map<?, ?> originals,  Map<String, ?>
 
         this.originals = resolveConfigVariables(configProviderProps, (Map<String, Object>) originals);
         this.values = definition.parse(this.originals);
-        this.used = Collections.synchronizedSet(new HashSet<>());
+        this.used = new CopyOnWriteArraySet<>();

Review comment:
       Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org