[
https://issues.apache.org/jira/browse/KAFKA-13689?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17511522#comment-17511522
]
Guozhang Wang commented on KAFKA-13689:
---------------------------------------
Hi [~RivenSun] I think I agree with you on the general note, my concern is that
since we do not control when `logUnused` is called, at the time when not all
provided values are retrieved, then we would log `config ... is not used.`
which would then be a bit misleading since they are likely going to be used
later indeed. But after a second thought, the semantics is fine we we just say
`config ... is not yet used.` at the time when `logUnused` is called, so it's
really the responsibility of the caller regarding when they want to call this
function to check which configs are not yet used.
So I think we can just like you said log it as (just adding `yet` at the end of
the sentence).
{code}
public void logUnused() {
Set<String> unusedkeys = unused();
if (!unusedkeys.isEmpty()) {
log.warn("These configurations '{}' were supplied but are not used
yet.", unusedkeys);
}
}
{code}
Since we always call `logUnused` at the end of the constructor of
producer/consumer/admin, then it's very likely that those unknown configs are
not retrieved yet and hence would be logged. For that, I'd say we update our
web docs indicating this effect exactly to clear any possible confusions.
> 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)