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

ASF GitHub Bot commented on KAFKA-7509:
---------------------------------------

rhauch opened a new pull request #5876: KAFKA-7509: Changed to avoid passing 
non-applicable properties to producer, consumer, and admin client
URL: https://github.com/apache/kafka/pull/5876
 
 
   The producer, consumer, and admin client log properties that are supplied 
but unused by the producer. Previously, Connect would pass many of its worker 
configuration properties into the producer, consumer, and admin client used for 
internal topics, resulting in lots of log warnings about unused config 
properties.
   
   With this change, Connect attempts to filter out the worker’s configuration 
properties that are not applicable to the producer, consumer, or admin client 
used for _internal_ topics. (Connect is already including only producer and 
consumer properties when creating those clients for connectors, since those 
properties are prefixed in the worker config.)
   
   For the most part, this is relatively straightforward, since there are some 
top-level worker-specific properties that can be removed, and most 
extension-specific properties have Connect-specific properties. Unfortunately, 
the REST extension is the only type of connect extension that uses unprefixed 
properties from the worker config, so any it is not possible to remove those 
from the properties passed to the producer, consumer, and admin clients. 
Hopefully, REST extensions are prevalant yet, and this will mean most users may 
not see any warnings about unused properties in the producer, consumer, and 
admin client.
   
   Removing the Connect worker configs is one step. The other is for the 
producer to remove the any properties that are specific to the consumer and 
admin client. Likewise, for the consumer we have to remove any properties that 
are specific to the producer and admin client, and for the admin client remove 
any properties that are specific to the producer and consumer. Note that any 
property that is unknown (e.g., properties for REST extension, interceptors, 
metric reporters, serdes, partitioners, etc.) must be passed to the producer, 
consumer, and admin client. All of these — except for the REST extension 
properties — should indeed be used by the producer and consumer. But, since the 
admin client only supports metric reporters, any properties for interceptors, 
serdes, partitioners and REST extension will also be logged as unused. This is 
about the best we can do at this point.
   
   All of this filtering logic was added to the `ConnectUtils` class, allowing 
the logic to be easily unit tested. All changes are limited to Kafka Connect, 
and will work with all client and Connect extensions (passing them to the 
clients if they are unknown).
   
   This supersedes #5867 and #5802.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   

----------------------------------------------------------------
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


> 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: clients, 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)

Reply via email to