-----BEGIN PGP SIGNED MESSAGE-----
Hash: SHA512

I am just getting aware of this KIP (not sure why I missed it).

In Kafka Streams we have nested clients and need to "forward" configs
from outer layer to inner layers -- hence, we prefix some configs to
be able to know which inner nested clients needs this config.

I think the simplest approach is, to add a prefix (like
"userconfig."). All thus configs would be skipped in the validation
step to avoid the WARN log.

When forwarding configs to inner classed (like nested clients in KS,
serializers etc) we would remove this prefix).

Using a `RecordingMap` seem rather heavy weight and complex?

Thoughts?

- -Matthias

On 2/17/20 9:09 AM, John Roesler wrote:
> Thanks Patrik,
>
> This seems to be a long and wandering issue. It seems that
> KAFKA-7509 has followed a similar trajectory to KAFKA-6793/KIP-552
> , and 7509 is just recently closed in favor of whatever we decide
> to do in KAFKA-6793.
>
> Considering (what I hope is) the whole history of this issue, a few
> things emerge:
>
> 1. It's useful to get warned when you pass an invalid
> configuration 2. It's not possible for the "top layer" (Streams,
> Connect, etc.) to know up front which configurations are applicable
> to pass down to the "second" layer (Clients, RocksDB) because those
> layers themselves are extensible (see below) 3. We should propose a
> change that fixes this issue for the whole Kafka ecosystem at
> once.
>
> Elaboration on point 2: Users of Kafka libraries need to register
> extra components like Processors, Interceptors,
> RocksDBConfigSetters, RebalanceListeners, etc. They need to pass
> configurations into these self-registered components. Therefore,
> the outermost component (the one that you directly pass a
> Properties to, and that instantiates other Configurable
> components) _cannot_ know which configurations are needed by the
> "extra" components inside the Configurable components. Therefore,
> no approach that involves filtering only the "needed"
> configurations up front, before constructing a Configurable
> component, could work.
>
> Randall made an aside in this comment:
> https://issues.apache.org/jira/browse/KAFKA-7509?focusedCommentId=1667
3834&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpan
el#comment-16673834
>
>
which I think is the most promising path right now.
> Namely, to use RecordingMap (or a similar approach) when
> configuring internal components and finally warn when _everything_
> has been wired up if some configuration value wasn't used by _any_
> component.
>
> It seems like this approach would satisfy all three of the above
> points, but it needs some design/discovery work to see what gaps
> exist in the current code base to achieve the goal. It also might
> be a fair amount of work (which is why we didn't follow that
> approach in KAFKA-7509), but I don't think there have been any
> other suggestions that satisfy both point 1 and point 2.
>
> Thoughts? -John
>
> On Wed, Feb 12, 2020, at 02:07, Patrik Kleindl wrote:
>> Hi John
>>
>> Regarding Kafka Streams this can probably be fixed easily, but it
>> does not handle the underlying issue that other custom prefixes
>> are not supported. Seems I even did a short analysis several
>> months ago and forgot about it, see
>> https://issues.apache.org/jira/browse/KAFKA-6793?focusedCommentId=168
70899&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpa
nel#comment-16870899
>>
>>
>>
We have used custom prefixes to pass properties to the RocksDBConfigSett
er
>> and it seems people are doing something similar in Connect, see
>> https://issues.apache.org/jira/browse/KAFKA-7509
>>
>> This KIP just seeks to avoid the false positives and setting it
>> to debug was preferred over implementing the custom prefixes.
>>
>> best regards
>>
>> Patrik
>>
>> On Tue, 11 Feb 2020 at 18:21, John Roesler <vvcep...@apache.org>
>> wrote:
>>
>>> Ah... I've just looked at some integration tests in Streams,
>>> and see the same thing.
>>>
>>> I need to apologize to everyone in the thread for my lack of
>>> understanding, and to thank Gwen for her skepticism. Looking
>>> back at the KIP itself, I see that Artur specifically listed
>>> log messages caused by Streams itself, which I failed to
>>> realize shouldn't be there at all.
>>>
>>> It now seems that we should not have a KIP at all, and also
>>> shouldn't make any changes to log levels or loggers. Instead we
>>> should treat KAFKA-6793 as a normal bug whose cause is that
>>> Streams does not correctly construct the client configurations
>>> when initializing the clients. It is leaving in the prefixed
>>> version of the client configs, but it should remove them. We
>>> should also add a test that we can specify all kinds of client
>>> configurations to Streams and that no WARN logs result during
>>> startup.
>>>
>>> Artur, what do you think about cancelling KIP-552 and instead
>>> just implementing a fix?
>>>
>>> Again, I'm really sorry for not realizing this sooner. And
>>> again, thanks to Gwen for chiming in.
>>>
>>> -John
>>>
>>> On Mon, Feb 10, 2020, at 02:19, Patrik Kleindl wrote:
>>>> Hi John Starting an empty streams instance
>>>>
>>>> final String bootstrapServers = "broker0:9092"; Properties
>>>> streamsConfiguration = new Properties();
>>>> streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG,
>>>
>>>>
"configDemo");
>>>> streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG,
>>>>
>>>>
bootstrapServers);
>>>> StreamsBuilder builder = new StreamsBuilder(); final
>>>> KafkaStreams streams = new KafkaStreams(builder.build(),
>>>> streamsConfiguration); streams.start();
>>>>
>>>> results in:
>>>>
>>>> stream-thread
>>>> [configDemo-bcaf82b4-324d-4956-a2a8-1dea0a8e3a2e-StreamThread-1]
>>>>
>>>>
Creating consumer client
>>>> ConsumerConfig values: ... stream-thread
>>>> [configDemo-bcaf82b4-324d-4956-a2a8-1dea0a8e3a2e-StreamThread-1-con
sumer]
>>>>
>>>>
Cooperative rebalancing enabled now
>>>> The configuration 'admin.retries' was supplied but isn't a
>>>> known config. The configuration 'admin.retry.backoff.ms' was
>>>> supplied but isn't a known config. Kafka version: 2.4.0
>>>>
>>>> when the normal consumer is created, but not for admin client
>>>> / producer / restore consumer.
>>>>
>>>> StreamsConfig seems to include this on purpose:
>>>>
>>>> final AdminClientConfig adminClientDefaultConfig = new
>>>> AdminClientConfig(getClientPropsWithPrefix(ADMIN_CLIENT_PREFIX,
>>>>
>>>>
AdminClientConfig.configNames()));
>>>> consumerProps.put(adminClientPrefix(AdminClientConfig.RETRIES_CONFI
G),
>>>>
>>>>
adminClientDefaultConfig.getInt(AdminClientConfig.RETRIES_CONFIG));
>>>>
>>> consumerProps.put(adminClientPrefix(AdminClientConfig.RETRY_BACKOFF_
MS_CONFIG),
>>>>
>>>
>>>
adminClientDefaultConfig.getLong(AdminClientConfig.RETRY_BACKOFF_MS_CONF
IG));
>>>>
>>>> If I add
>>>>
>>>>
>>> streamsConfiguration.put(StreamsConfig.restoreConsumerPrefix(Consume
rConfig.RECEIVE_BUFFER_CONFIG),
>>>>
>>>
65536);
>>>>
>>> streamsConfiguration.put(StreamsConfig.mainConsumerPrefix(ConsumerCo
nfig.MAX_POLL_RECORDS_CONFIG),
>>>>
>>>
100);
>>>>
>>>> then the warnings
>>>>
>>>> The configuration 'main.consumer.max.poll.records' was
>>>> supplied but isn't a known config. The configuration
>>>> 'restore.consumer.receive.buffer.bytes' was supplied but
>>>> isn't a known config.
>>>>
>>>> are shown for all clients, not only the last consumer.
>>>>
>>>> Streams provides these prefixes so maybe they are not
>>>> handled correctly regarding the log message.
>>>>
>>>> Maybe this helps to pinpoint the source of this in KS at
>>>> least
>>>>
>>>> best regards
>>>>
>>>> Patrik
>>>>
>>>>
>>>> On Sat, 8 Feb 2020 at 05:11, John Roesler
>>>> <vvcep...@apache.org> wrote:
>>>>
>>>>> Looking at where the log message comes from:
>>>>> org.apache.kafka.common.config.AbstractConfig#logUnused it
>>>>> seems like maybe the warning just happens when you pass
>>>>> extra configs to a client that it has no knowledge of (and
>>>>> therefore doesn't "use").
>>>>>
>>>>> I'm now suspicious if Streams is actually sending extra
>>>>> configs to the clients, although it seems like we _don't_
>>>>> see these warnings in other cases.
>>>>>
>>>>> Maybe some of the folks who actually see these messages can
>>>>> try to
>>> pinpoint
>>>>> where exactly the rogue configs are coming from?
>>>>>
>>>>> I might have overlooked a message at some point, but it
>>>>> wasn't clear to me that we were talking about warnings that
>>>>> were actually caused by Streams. I thought the unknown
>>>>> configs were something user-specified.
>>>>>
>>>>> Thanks, -John
>>>>>
>>>>> On Fri, Feb 7, 2020, at 13:10, Gwen Shapira wrote:
>>>>>> Ah, got it! I am indeed curious why they do this :)
>>>>>>
>>>>>> Maybe John can shed more light. But if we can't find a
>>>>>> better fix, perhaps the nice thing to do is really a
>>>>>> separate logger, so users
>>> who
>>>>>> are not worried about shooting themselves in the foot can
>>>>>> make those warnings go away.
>>>>>>
>>>>>> Gwen Shapira Engineering Manager | Confluent 650.450.2760
>>>>>> | @gwenshap Follow us: Twitter | blog
>>>>>>
>>>>>> On Fri, Feb 07, 2020 at 4:13 AM, Patrik Kleindl <
>>>>>> pklei...@gmail.com
>>>>
>>>>> wrote:
>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> Hi Gwen
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> Kafka Streams is not a third party library and produces
>>>>>>> a lot of
>>> these
>>>>>>> warnings, e.g.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> *The configuration 'main.consumer.max.poll.records' was
>>>>>>> supplied
>>> but
>>>>> isn't
>>>>>>> a known config.* *The configuration 'admin.retries' was
>>>>>>> supplied but isn't a known
>>>>> config.*
>>>>>>> and various others if you try to fine-tune the
>>>>>>> restoration
>>> consumer or
>>>>>>> inject parameters for state stores. This results in a
>>>>>>> lot of false positives and only makes new people
>>>>> worried
>>>>>>> and then ignore the warnings altogether.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> Unless this is taken care of at least the Kafka Streams
>>>>>>> users will probably be better off having this on debug
>>>>>>> level.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> Best regards
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> Patrik
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Thu, 6 Feb 2020 at 16:55, Gwen Shapira < gwen@
>>>>>>> confluent. io ( g...@confluent.io ) > wrote:
>>>>>>>
>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> INFO is the default log level, and while it looks
>>>>>>>> less "alarming"
>>> than
>>>>>>>> WARN, users will still see it and in my experience,
>>>>>>>> they will
>>> worry
>>>>> that
>>>>>>>> something is wrong anyway. Or if INFO isn't the
>>>>>>>> default, users
>>> won't
>>>>> see
>>>>>>>> it, so it is no different from debug and we are left
>>>>>>>> with no way
>>> of
>>>>>>>> warning users that they misconfigured something.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> The point is that "known configs" exist in Kafka as a
>>>>>>>> validation
>>>>> step. It
>>>>>>>> is there to protect users. So anything that makes the
>>>>>>>> concerns
>>> about
>>>>>>>> unknown configs invisible to users, makes the
>>>>>>>> validation step
>>> useless
>>>>> and
>>>>>>>> we may as well remove it. I'm against that - I think
>>>>>>>> users should
>>> be
>>>>> made
>>>>>>>> aware of misconfigs as much as possible - especially
>>>>>>>> since if you
>>>>> misspell
>>>>>>>>
>>>>>>>> "retention", you will lose data.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> If we look away from the symptom and go back to the
>>>>>>>> actual
>>> cause....
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> I think Kafka had a way (and maybe it still does) for
>>>>>>>> 3rd party
>>>>> developers
>>>>>>>> who create client plugins (mostly interceptors) to
>>>>>>>> make their
>>> configs
>>>>>>>> "known". 3rd party developers should be responsible
>>>>>>>> for the good experience of their users. Now it is
>>>>>>>> possible that you'll pick a
>>> 3rd
>>>>> party
>>>>>>>> library that didn't do it and have a worse user
>>>>>>>> experience, but I
>>> am
>>>>> not
>>>>>>>> sure it is the job of Apache Kafka to protect users
>>>>>>>> from their
>>> choice
>>>>> of
>>>>>>>> libraries (and as long as those libraries are OSS,
>>>>>>>> users can fix them).
>>>>> Especially
>>>>>>>> not at the expense of someone who doesn't use 3rd
>>>>>>>> party libs.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> Gwen
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> Gwen Shapira Engineering Manager | Confluent
>>>>>>>> 650.450.2760 | @gwenshap Follow us: Twitter | blog
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Feb 06, 2020 at 2:06 AM, Artur Burtsev <
>>>>>>>> artjock@ gmail.
>>> com
>>>>> (
>>>>>>>> artj...@gmail.com ) > wrote:
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Hi John,
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> In out case it wont help, since we are running
>>>>>>>>> instance per
>>>>> partition and
>>>>>>>>> even with summary only we get 32 warnings per
>>>>>>>>> rollout.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Hi Gwen,
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Thanks for you reply, I understand and share your
>>>>>>>>> concern, I also mentioned it earlier in the thread.
>>>>>>>>> Do you think it will work if
>>> we
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> change
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> DEBUG to INFO?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Thanks, Artur
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Thu, Feb 6, 2020 at 4:21 AM Gwen Shapira < gwen@
>>>>>>>>> confluent.
>>> io (
>>>>> gwen@ confluent.
>>>>>>>>> io ( g...@confluent.io ) ) > wrote:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Sorry for late response. The reason that unused
>>>>>>>>>> configs is in
>>> WARN
>>>>> is
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> that
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> if you misspell a config, it means that it will
>>>>>>>>>> not apply. In
>>> some
>>>>> cases
>>>>>>>>>> (default retention) you want know until too late.
>>>>>>>>>> We wanted to
>>> warn
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> admins
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> about possible misconfigurations.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> In the context of a company supporting Kafka -
>>>>>>>>>> customers run
>>> logs at
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> INFO
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> level normally, so if we suspect a
>>>>>>>>>> misconfiguration, we don't
>>> want
>>>>> to
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> ask
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> the customer to change level to DEBUG and bounce
>>>>>>>>>> the broker. It
>>> is
>>>>> time
>>>>>>>>>> consuming and can be risky.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> *Gwen Shapira* Product Manager | Confluent
>>>>>>>>>> 650.450.2760 | @gwenshap Follow us: Twitter (
>>>>>>>>>> https:/ / twitter. com/ ConfluentInc (
>>> https:/
>>>>> / twitter.
>>>>>>>>>> com/ ConfluentInc (
>>>>>>>>>> https://twitter.com/ConfluentInc ) ) ) |
>>> blog
>>>>> ( http:/
>>>>>>>>>> / www. confluent.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> io/
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> blog ( http:/ / www. confluent. io/ blog (
>>>>> http://www.confluent.io/blog ) )
>>>>>>>>>> )
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Sent via Superhuman ( https:/ / sprh. mn/
>>>>>>>>>> ?vip=gwen@
>>> confluent. io
>>>>> ( https:/
>>>>>>>>>> / sprh. mn/ ?vip=gwen@ confluent. io (
>>>>>>>>>> https://sprh.mn/?vip=g...@confluent.io ) ) )
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Mon, Jan 06, 2020 at 4:21 AM, Stanislav
>>>>>>>>>> Kozlovski <
>>> stanislav@
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> confluent.
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> io ( stanislav@ confluent. io (
>>>>>>>>>> stanis...@confluent.io ) ) >
>>> wrote:
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Hey Artur,
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Perhaps changing the log level to DEBUG is the
>>>>>>>>>>> simplest
>>> approach.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> I wonder if other people know what the
>>>>>>>>>>> motivation behind the
>>> WARN
>>>>> log
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> was?
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> I'm struggling to think up of a scenario where
>>>>>>>>>>> I'd like to see
>>>>> unused
>>>>>>>>>>> values printed in anything above DEBUG.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Best, Stanislav
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Mon, Dec 30, 2019 at 12:52 PM Artur Burtsev
>>>>>>>>>>> < artjock@
>>> gmail.
>>>>> com
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> ( artjock@
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> gmail. com ( artjock@ gmail. com (
>>>>>>>>>>> artj...@gmail.com ) ) ) >
>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Hi,
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Indeed changing the log level for the whole
>>>>>>>>>>>> AbstractConfig is
>>> not
>>>>> an
>>>>>>>>>>>> option, because logAll is extremely useful.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Grouping warnings into 1 (with the count of
>>>>>>>>>>>> unused only) will
>>> not
>>>>> be a
>>>>>>>>>>>> good option for us either. It will still be
>>>>>>>>>>>> pretty noisy.
>>> Imagine
>>>>> we
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> have
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 32 partitions and scaled up the application
>>>>>>>>>>>> to 32 instances
>>> then
>>>>> we
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> still
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> have 32 warnings per application (instead of
>>>>>>>>>>>> 96 now) while we
>>>>> would
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> like
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> to have 0 warnings because we are perfectly
>>>>>>>>>>>> aware of using schema.registry.url and its
>>>>>>>>>>>> totally fine, and we don't have
>>> to be
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> warned
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> every time we start the application. Now
>>>>>>>>>>>> imagine we use more
>>> than
>>>>> one
>>>>>>>>>>>> consumer per application, then it will add
>>>>>>>>>>>> another
>>> multiplication
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> factor
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> to these grouped warnings and we still have a
>>>>>>>>>>>> lot of those.
>>> So I
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> would say
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> grouping doesn't help much.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> I think adding extra logger like
>>>>>>>>>>>> "org.apache.kafka.clients.producer.ProducerConfig.unused"
>>>
>>>>>>>>>>>>
could be
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> another
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> good option. That would leave the existing
>>>>>>>>>>>> interface
>>> untouched and
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> give
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> everyone an option to mute irrelevant
>>>>>>>>>>>> warnings.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> To summarize, I still can see 3 options with
>>>>>>>>>>>> its pros and cons
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> discussed
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> in the thread: 1) extra config with interface
>>>>>>>>>>>> to handle unused 2) change unused warn to
>>>>>>>>>>>> debug 3) add extra logger for unused
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Please let me know what do you think.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks, Artur
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Mon, Dec 30, 2019 at 11:07 AM Stanislav
>>>>>>>>>>>> Kozlovski < stanislav@ confluent. io (
>>>>>>>>>>>> stanislav@ confluent. io (
>>>>> stanislav@ confluent.
>>>>>>>>>>>> io ( stanis...@confluent.io ) ) ) > wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Hi all,
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Would printing all the unused
>>>>>>>>>>>>> configurations in one line,
>>> versus
>>>>> N
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> lines,
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> be more helpful? I know that it would
>>>>>>>>>>>>> greatly reduce the
>>>>> verbosity
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> in log
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> visualization tools like Kibana while still
>>>>>>>>>>>>> allowing us to
>>> see
>>>>> all
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> the
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> relevant information without the need for
>>>>>>>>>>>>> an explicit action
>>> (e.g
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> changing
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> the log level)
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Best, Stanislav
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Sat, Dec 28, 2019 at 3:13 PM John
>>>>>>>>>>>>> Roesler < vvcephei@
>>> apache.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> org ( vvcephei@
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> apache. org ( vvcephei@ apache. org (
>>>>>>>>>>>>> vvcep...@apache.org )
>>> ) )
>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi Artur,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> That’s a good point.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> One thing you can do is log a summary at
>>>>>>>>>>>>>> WARN level, like
>>> “27
>>>>>>>>>>>>>> configurations were ignored. Ignored
>>>>>>>>>>>>>> configurations are
>>> logged
>>>>> at
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> DEBUG
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> level.”
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I looked into the code a little, and
>>>>>>>>>>>>>> these log messages are
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> generated
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> in
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> AbstractConfig (logAll and logUnused).
>>>>>>>>>>>>>> They both use the
>>> logger
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> associated
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> with the relevant config class
>>>>>>>>>>>>>> (StreamsConfig,
>>> ProducerConfig,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> etc.).
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> The
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> list of all configs is logged at INFO
>>>>>>>>>>>>>> level, and the list of
>>>>> unused
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> configs
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> is logged at WARN level. This means that
>>>>>>>>>>>>>> it's not possible
>>> to
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> silence
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> the
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> unused config messages while still
>>>>>>>>>>>>>> logging the list of all
>>>>> configs.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> You
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> could only silence both by setting (for
>>>>>>>>>>>>>> example)
>>> ProducerConfig
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> logger
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> to
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ERROR or OFF.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> If it's desirable to be able to toggle
>>>>>>>>>>>>>> them independently,
>>> then
>>>>> you
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> can
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> create a separate logger for unused
>>>>>>>>>>>>>> configs, named something
>>>>> like
>>>>>>>>>>>>>> "org.apache.kafka.clients.producer.ProducerConfig.unused"
.
>>>>>
>>>>>>>>>>>>>>
Then, you
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> can
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> leave the log at WARN, so it would
>>>>>>>>>>>>>> continue to be printed by
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> default,
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> and
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> anyone could disable it by setting
>>>>>>>>>>>>>> "org.apache.kafka.clients.producer.ProducerConfig.unused"
>>>>>>>>>>>>>> to
>>>>> ERROR
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> or
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> OFF,
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> without disturbing the rest of the config
>>>>>>>>>>>>>> log messages.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> It's simpler without the extra logger,
>>>>>>>>>>>>>> but you also get less
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> control.
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Do
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> you think the extra control is necessary,
>>>>>>>>>>>>>> versus printing a
>>>>> summary
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> at
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> WARN
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> level? -John
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Fri, Dec 27, 2019, at 04:26, Artur
>>>>>>>>>>>>>> Burtsev wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Indeed changing log level to debug
>>>>>>>>>>>>>>> would be the easiest
>>> and I
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> think that
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> would be a good solution. When no one
>>>>>>>>>>>>>>> object I'm ready to
>>> move
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> forward
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> with this approach and submit a MR.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The only minor thing I have – having it
>>>>>>>>>>>>>>> at debug log level
>>>>> might
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> make it a
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> bit less friendly for developers,
>>>>>>>>>>>>>>> especially for those who
>>>>> just do
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> the
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> first steps in Kafka. For example, if
>>>>>>>>>>>>>>> you misspelled the
>>>>> property
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> name and
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> trying to understand why things don't
>>>>>>>>>>>>>>> do what you expect.
>>>>> Having a
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> warning
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> might save some time in this case.
>>>>>>>>>>>>>>> Other
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> than
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> that I cannot see any reasons to have
>>>>>>>>>>>>>>> warnings there.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks, Artur
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, Dec 26, 2019 at 10:01 PM John
>>>>>>>>>>>>>>> Roesler < vvcephei@
>>>>> apache.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> org ( vvcephei@
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> apache. org ( vvcephei@ apache. org (
>>>>>>>>>>>>>>> vvcep...@apache.org
>>> ) )
>>>>> ) >
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks for the KIP, Artur!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> For reference, here is the kip:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> https:/ / cwiki. apache. org/ confluence/
>>>>>>>>>>>> display/ KAFKA/
>>>>>>>>>>>> KIP-552%3A+Add+interface+to+handle+unused+config
>>>>>>>>>>>>
>>>>>>>>>>>>
(
>>>>>>>>>>>> https:/ / cwiki. apache. org/ confluence/
>>>>>>>>>>>> display/ KAFKA/
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> KIP-552%3A+Add+interface+to+handle+unused+config
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> (
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> https:/ / cwiki. apache. org/ confluence/ display/
>>>>>>>> KAFKA/
>>>>> KIP-552%3A+Add+interface+to+handle+unused+config
>>>>>>>> (
>>>>>>>>
>>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-552%3A+Add+int
erface+to+handle+unused+config
>>>>>>>>
>>>
)
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> ) )
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I agree, these warnings are kind of a
>>>>>>>>>>>>>>>> nuisance. Would it
>>> be
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> feasible
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> just to leverage log4j in some way to
>>>>>>>>>>>>>> make it easy to filter
>>>>> these
>>>>>>>>>>>>>> messages? For example, we could move
>>>>>>>>>>>>>> those warnings to debug
>>>>> level,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> or
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> even
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> use a separate logger for them.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks for starting the discussion.
>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Tue, Dec 24, 2019, at 07:23, Artur
>>>>>>>>>>>>>>>> Burtsev wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> This KIP provides a way to deal
>>>>>>>>>>>>>>>>> with a warning "The
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> configuration {}
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> was supplied but isn't a known
>>>>>>>>>>>>>>>>> config." when it is not
>>>>> relevant.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Cheers, Artur
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> -- Best, Stanislav
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> -- Best, Stanislav
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>
>>>>
>>>
>>
-----BEGIN PGP SIGNATURE-----

iQIzBAEBCgAdFiEEI8mthP+5zxXZZdDSO4miYXKq/OgFAl5K2L0ACgkQO4miYXKq
/Oj9+A//bovjqaGV83FeY6wWbVHRjrHPreTE22xTjek2ebfUaPCEBCzHXtVogJQF
T487hkePlJayWQU95Qq55fkmOTA6gVL5jLoyXmzipiwhWOJjopTOXSaxaJgpZWu6
Q3A0L1EUF5iH5eJ6u8ST644UZmH+lSYtXkQn3/xSqgfT+F72EFEeCIlN2bcL0Bi4
46MV5AEqgyxz5k+E9sl0eGpqnSJBRBnk7n0opWgQ82vO8KluCUFOuu7h1dc8Yc7R
olDIucl9sNPs2CTflTc6J9176YCKq/sEnnWmSSBrQVVmSRdPabjW6Uy7Q/8wmtlN
r1K6VkaIypB7M36rT2Y4jZgI890P7o2FmuTCTfmpogzYdf1pedqaFKJ0cpx0jUjz
bpxJ13vxWWLBE588Bx/41jSDNhxprTuS0Hbu0xh6yDWxXeM7aWESQTdm6bhXHfG+
5AFRKa+NAUhXjfjX6kH8TcK1QXtYn48R9XukBySORHZRZhv6Tb3WogeznsldVzsL
K9GoKxYCeMDbPL6JDuQMBBenzbd40rRbMtzVs2pR/jZUU6vhRPPfVQAA++n6agNP
e/F86AZZfQ3RMMZ4th/eRteFX85GoSDrLcg6C5p7JXFOR9y3eXXSC3/y8OkyzjRG
Jz9H3pwKUf73POZJGBbjWPw/FmbBwp0TMy+bZfdvN7z7FyaRI4A=
=tkBz
-----END PGP SIGNATURE-----

Reply via email to