[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16672075#comment-16672075 ]
Randall Hauch commented on KAFKA-7509: -------------------------------------- Okay, I am now to the point where I think this "hygiene" approach is completely flawed, and that instead the problem is that we're logging as warnings all "unused" properties in the producer, consumer, and the KafkaAdminClient. Let me back up. Based upon the discussion above, I modified my approach to attempt to retain all of the configuration properties that are known to the ProducerConfig, ConsumerConfig, and AdminClientConfig, where "known" properties include: * all of those whose name is in the set returned by each of the config's configNames(), or * any property that can be passed to an interceptor, key or value (de)serializer, metrics reporter, or partitioner that is instantiated by the client. That last bit is the problem: the properties to the clients' interceptors, serdes, metrics reporter, and partitions are all unprefixed, so it is impossible to know which properties are needed by any of the specified implementations. IOW, the properties passed to a producer, consumer, or admin client must be able to include any property that is needed by any of these custom components. And, because the getConfiguredComponent method used by the clients passes the Map<String, Object> to the component's configure method, the AbstractConfig doesn't know whether those properties are even used by the component. So, if the AbstractConfig doesn't really even know whether it a property is really used or unused, why are the Producer, Consumer, and KafkaAdminClient even bothering to log "unused" properties? h3. Bottom line I now posit that the only way to accurately eliminate these warnings is to remove the config.logUnused() call from the Producer, Consumer, and KafkaAdminClient, or to change AbstractConfig.logUnused() to log these at the INFO (or DEBUG) level. > Kafka Connect logs unnecessary warnings about unused configurations > ------------------------------------------------------------------- > > Key: KAFKA-7509 > URL: https://issues.apache.org/jira/browse/KAFKA-7509 > Project: Kafka > Issue Type: Improvement > Components: KafkaConnect > Affects Versions: 0.10.2.0 > Reporter: Randall Hauch > Assignee: Randall Hauch > Priority: Major > > When running Connect, the logs contain quite a few warnings about "The > configuration '{}' was supplied but isn't a known config." This occurs when > Connect creates producers, consumers, and admin clients, because the > AbstractConfig is logging unused configuration properties upon construction. > It's complicated by the fact that the Producer, Consumer, and AdminClient all > create their own AbstractConfig instances within the constructor, so we can't > even call its {{ignore(String key)}} method. > See also KAFKA-6793 for a similar issue with Streams. > There are no arguments in the Producer, Consumer, or AdminClient constructors > to control whether the configs log these warnings, so a simpler workaround > is to only pass those configuration properties to the Producer, Consumer, and > AdminClient that the ProducerConfig, ConsumerConfig, and AdminClientConfig > configdefs know about. -- This message was sent by Atlassian JIRA (v7.6.3#76005)