[ https://issues.apache.org/jira/browse/KAFKA-2289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16301901#comment-16301901 ]
ASF GitHub Bot commented on KAFKA-2289: --------------------------------------- guozhangwang closed pull request #71: KAFKA-2289: KafkaProducer logs erroneous warning on startup URL: https://github.com/apache/kafka/pull/71 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/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 5a37580ec69..4f10cc89bcf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -256,6 +256,7 @@ private KafkaProducer(ProducerConfig config, Serializer<K> keySerializer, Serial Serializer.class); this.keySerializer.configure(config.originals(), true); } else { + config.use(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG); this.keySerializer = keySerializer; } if (valueSerializer == null) { @@ -263,6 +264,7 @@ private KafkaProducer(ProducerConfig config, Serializer<K> keySerializer, Serial Serializer.class); this.valueSerializer.configure(config.originals(), false); } else { + config.use(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG); this.valueSerializer = valueSerializer; } config.logUnused(); diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index bae528d3151..642774760e7 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -51,10 +51,14 @@ public AbstractConfig(ConfigDef definition, Map<?, ?> originals) { } protected Object get(String key) { + use(key); + return values.get(key); + } + + public void use(String key) { if (!values.containsKey(key)) throw new ConfigException(String.format("Unknown configuration '%s'", key)); used.add(key); - return values.get(key); } public Short getShort(String key) { ---------------------------------------------------------------- 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 > KafkaProducer logs erroneous warning on startup > ----------------------------------------------- > > Key: KAFKA-2289 > URL: https://issues.apache.org/jira/browse/KAFKA-2289 > Project: Kafka > Issue Type: Bug > Components: clients > Affects Versions: 0.8.2.1 > Reporter: Henning Schmiedehausen > Priority: Trivial > > When creating a new KafkaProducer using the > KafkaProducer(KafkaConfig, Serializer<K>, Serializer<V>) constructor, Kafka > will list the following lines, which are harmless but are still at WARN level: > WARN [2015-06-19 23:13:56,557] > org.apache.kafka.clients.producer.ProducerConfig: The configuration > value.serializer = class XXXX was supplied but isn't a known config. > WARN [2015-06-19 23:13:56,557] > org.apache.kafka.clients.producer.ProducerConfig: The configuration > key.serializer = class YYYY was supplied but isn't a known config. -- This message was sent by Atlassian JIRA (v6.4.14#64029)