KAFKA-10090 Misleading warnings: The configuration was supplied but i…#8826
KAFKA-10090 Misleading warnings: The configuration was supplied but i…#8826chia7712 merged 7 commits intoapache:trunkfrom
Conversation
abbccdda
left a comment
There was a problem hiding this comment.
One meta comment, in the ticket we stated that we want to fix the warning for used instead of unknown. Does it make sense to just change the logging? And in terms of misleading, does this log confuse the user by any chance?
|
@chia7712 : Thanks for the PR. I ran the following command with the PR. kafka.properties I still saw the WARN.
|
|
@junrao Thanks for your response. fixed. |
|
@chia7712 : Thanks for the updated PR. Did that fix the issue since I still saw the same WARN when running kafka-topics.sh. |
my bad :( |
There was a problem hiding this comment.
Does this cover the case when listenerName is not null? I guess that can only happen on the server side and since we don't log unused configs on the server, so maybe this is ok for now?
There was a problem hiding this comment.
if (listenerName == null)
parsedConfigs = (Map<String, Object>) config.values();
else
parsedConfigs = config.valuesWithPrefixOverride(listenerName.configPrefix());the method config.valuesWithPrefixOverride also returns ```RecordingMap so it is ok.
b510a38 to
cb0aa27
Compare
There was a problem hiding this comment.
I tried running console-producer with/without this PR. It doesn't seem to WARN any unused SSL configs in either test. Do you know why?
@junrao this is the root cause.
There was a problem hiding this comment.
Hmm, why is this necessary since we reset used to empty in the next line?
There was a problem hiding this comment.
ConfigDef#parse (https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java#L468) tries to get all elements from input maps so all gets are recorded. In order to avoid recording, we pass a copy instead of RecordingMap.
There was a problem hiding this comment.
Ok. I guess the issue is in the following, where we pass in a RecordingMap to construct ProducerConfig.
However, that code seems no longer necessary since we are now setting clientId in ProducerConfig.postProcessParsedConfig(). Could we just avoid constructing ProducerConfig there?
There was a problem hiding this comment.
However, that code seems no longer necessary since we are now setting clientId in ProducerConfig.postProcessParsedConfig(). Could we just avoid constructing ProducerConfig there?
I don't think so. The configs passed to configurable object is origins so the generated "client.id" is not included. However, your feedback inspires me that we don't need to create a new ProducerConfig. Instead, we can use overrideConfig to set generated client.id to those configurable object. Will update it later.
There was a problem hiding this comment.
Hmm, I am still a bit confused. My understanding is that with the latest change, ProducerConfig will only be instantiated once and thus the passed in originals will never be a RecordingMap. But it seems this is still needed? Could you explain a bit more why this is the case?
There was a problem hiding this comment.
But it seems this is still needed?
It is not necessary with the latest change. I kept it as a total solution (if someone pass RecordingMap in the future). However, I'm going to remove it to make this PR simpler.
unrelated error. rebase to trigger QA |
cb0aa27 to
2f5d4e3
Compare
There was a problem hiding this comment.
Hmm, why is this necessary since we reset used to empty in the next line?
There was a problem hiding this comment.
When will the input configs not be recording?
There was a problem hiding this comment.
Currently, the use cases of non-RecordingMap happens on tests. However, it seems to me we don't give a good definition of Configurable#configure. It is hard to say what we should pass to it. immutable map, mutable map and RecordingMap are alternatives. I want to keep flexibility but it is ok to me to rewrite related tests to make sure all pass are RecordingMap
There was a problem hiding this comment.
It tests the specify config is recorded when constructing KafkaConsumer
There was a problem hiding this comment.
Should we use the private static constructor in this class? Ditto below.
There was a problem hiding this comment.
It tests the specify config is recorded when constructing KafkaProducer
2f5d4e3 to
648fdc7
Compare
There was a problem hiding this comment.
Ok. I guess the issue is in the following, where we pass in a RecordingMap to construct ProducerConfig.
However, that code seems no longer necessary since we are now setting clientId in ProducerConfig.postProcessParsedConfig(). Could we just avoid constructing ProducerConfig there?
648fdc7 to
2415a0d
Compare
There was a problem hiding this comment.
Hmm, I am still a bit confused. My understanding is that with the latest change, ProducerConfig will only be instantiated once and thus the passed in originals will never be a RecordingMap. But it seems this is still needed? Could you explain a bit more why this is the case?
There was a problem hiding this comment.
Do we need to instantiate again?
There was a problem hiding this comment.
We need a new RecordingMap to test different key without listener prefix. Otherwise, the key may be used by previous test.
There was a problem hiding this comment.
Is this test necessary? Do we still have a case where we pass in a RecordingMap to ProducerConfig?
There was a problem hiding this comment.
I'm going to remove this test assertTrue(new ProducerConfig(config.originals(), false).unused().co...
|
@chia7712 : There are 6 failures in the system test run. Are they related to this PR? |
They are unrelated to this PR and I have opened a PR to fix them (#9673) |
…sn't a known config
|
rebase to trigger QA again |
…t-for-generated-requests * apache-github/trunk: MINOR: Fix flaky test shouldQueryOnlyActivePartitionStoresByDefault (apache#9681) KAFKA-10799 AlterIsr utilizes ReplicaManager ISR metrics (apache#9677) MINOR: Fix KTable-KTable foreign-key join example (apache#9683) KAFKA-10473: Add docs on partition size-on-disk, and other log-related metrics (apache#9276) KAFKA-10739; Replace EpochEndOffset with automated protocol (apache#9630) KAFKA-10460: ReplicaListValidator format checking is incomplete (apache#9326) KAFKA-10554; Perform follower truncation based on diverging epochs in Fetch response (apache#9382) MINOR: Align the UID inside/outside container (apache#9652) KAFKA-10794 Replica leader election is too slow in the case of too many partitions (apache#9675) KAFKA-10090 Misleading warnings: The configuration was supplied but i… (apache#8826) clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java core/src/test/scala/unit/kafka/server/epoch/util/ReplicaFetcherMockBlockingSend.scala
https://issues.apache.org/jira/browse/KAFKA-10090
Committer Checklist (excluded from commit message)