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

Guozhang Wang commented on KAFKA-13689:
---------------------------------------

Hi, I think we can first revisit the meaningfulness of the "logUnused" function 
itself: in "AbstractConfig" we only record "used" configs from those parsed 
definitions, and the `unused` set is the diff of the "originals" (note, not the 
"values") and the "used". This means:

1) Any pluggable component's configs, since they are not defined in the 
corresponding Producer/Consumer/etcConfig, they would always be in the "unused" 
set. The pluggable components thsmeslves would not try to use the "get" call to 
retrieve them anyways, but would usually us `originals().get()`. I.e. they are 
really just "unknown".
2) Any configs that are not called via "get" or "ignore" would be in this 
"unused" set. I.e. they are "unused".

The intention of "logUnused" function itself, is to warn people just about the 
second case (as this JIRA is trying to resolve), since assuming that the code 
is bug-free, this should NOT happen --- thinking about this, when certain 
component is not enabled, we should call `config.ignore(...)` on the 
corresponding configs, like `TRANSACTION_TIMEOUT_CONFIG` in this case. With 
that, we should not have "unused" configs at all. On the other hand, have a log 
line for each "unknown" config may be a bit too verbose, e.g. in Streams we 
have a bunch of pluggable component resulting in quite a lot of log lines for 
each of their custom config. I think this is not the original intention of the 
"logUnused" function to actually warn about the "unknown" configs at all, and 
it's less valuable to do so as well.

There's an additional scenario though, for deprecated configs: when we 
deprecate a config, the source code may not request them anymore but if the 
user does not change their code, they may still set values for those deprecated 
configs which would then end up in the `unused` set. So we should also call 
`config.ignore()` on those deprecated configs if they are no longer used inside 
the source code (note that to achieve backward compatibility, some deprecated 
configs may still be retrieved inside the code, in that case we would not need 
to call `ignore()`).

So if you agree with me that "logUnused" should really be a WARN for the second 
case above since this should never happen, I'd suggest we:

1) use "config.ignore" in cases when certain component are disabled and hence 
their corresponding configs would not be requested, or when the config is 
deprecated and no longer requested.
2) change the log line to "was supplied but isn't used" (don't we feel it's 
weird to log "it's not known" in a function called "logUnused"? :P).

> AbstractConfig log print information is incorrect
> -------------------------------------------------
>
>                 Key: KAFKA-13689
>                 URL: https://issues.apache.org/jira/browse/KAFKA-13689
>             Project: Kafka
>          Issue Type: Bug
>          Components: config
>    Affects Versions: 3.0.0
>            Reporter: RivenSun
>            Assignee: RivenSun
>            Priority: Major
>             Fix For: 3.2.0
>
>
> h1. 1.Example
> KafkaClient version is 3.1.0, KafkaProducer init properties:
>  
> {code:java}
> Properties props = new Properties();
> props.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, false);
> props.put(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, 60003);{code}
>  
>  
> Partial log of KafkaProducer initialization:
> {code:java}
>     ssl.truststore.location = C:\Personal 
> File\documents\KafkaSSL\client.truststore.jks
>     ssl.truststore.password = [hidden]
>     ssl.truststore.type = JKS
>     transaction.timeout.ms = 60003
>     transactional.id = null
>     value.serializer = class 
> org.apache.kafka.common.serialization.StringSerializer[main] INFO 
> org.apache.kafka.common.security.authenticator.AbstractLogin - Successfully 
> logged in.
> [main] WARN org.apache.kafka.clients.producer.ProducerConfig - The 
> configuration 'transaction.timeout.ms' was supplied but isn't a known config.
> [main] INFO org.apache.kafka.common.utils.AppInfoParser - Kafka version: 3.1.0
> [main] INFO org.apache.kafka.common.utils.AppInfoParser - Kafka commitId: 
> 37edeed0777bacb3
> [main] INFO org.apache.kafka.common.utils.AppInfoParser - Kafka startTimeMs: 
> 1645602332999 {code}
> From the above log, you can see that KafkaProducer has applied the user's 
> configuration, {*}transaction.timeout.ms=60003{*}, the default value of this 
> configuration is 60000.
> But we can see another line of log:
> [main] WARN org.apache.kafka.clients.producer.ProducerConfig - The 
> configuration *'transaction.timeout.ms'* was supplied but isn't a 
> *{color:#ff0000}known{color}* config.
>  
> h1. 2.RootCause:
> 1) ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG is set to {*}false{*}.
> So the configurations related to the KafkaProducer transaction will not be 
> requested.
> See the source code: KafkaProducer#configureTransactionState(...) .
> 2) AbstractConfig#logUnused() -> AbstractConfig#unused()
> {code:java}
> public Set<String> unused() {
>     Set<String> keys = new HashSet<>(originals.keySet());
>     keys.removeAll(used);
>     return keys;
> } {code}
> If a configuration has not been requested, the configuration will not be put 
> into the used variable. SourceCode see as below:
> AbstractConfig#get(String key)
>  
> {code:java}
> protected Object get(String key) {
>     if (!values.containsKey(key))
>         throw new ConfigException(String.format("Unknown configuration '%s'", 
> key));
>     used.add(key);
>     return values.get(key);
> } {code}
> h1.  
> h1. Solution:
> 1. AbstractConfig#logUnused() method
> Modify the log printing information of this method,and the unused 
> configuration log print level can be changed to {*}INFO{*}, what do you think?
> {code:java}
> /**
>  * Log infos for any unused configurations
>  */
> public void logUnused() {     for (String key : unused())
>         log.info("The configuration '{}' was supplied but isn't a used 
> config.", key);
> }{code}
>  
>  
> 2. AbstractConfig provides two new methods: logUnknown() and unknown()
> {code:java}
> /**
>  * Log warnings for any unknown configurations
>  */
> public void logUnknown() {
>     for (String key : unknown())
>         log.warn("The configuration '{}' was supplied but isn't a known 
> config.", key);
> } {code}
>  
> {code:java}
> public Set<String> unknown() {
>     Set<String> keys = new HashSet<>(originals.keySet());
>     keys.removeAll(values.keySet());
>     return keys;
> } {code}
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to