[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17519748#comment-17519748 ] Chris Egerton commented on KAFKA-7509: -- Thanks [~mjsax]. It seems like the root causes of this issue and KAFKA-6793 are actually quite different and, although we shouldn't implement a solution for one that is wholly incompatible with a solution for the other, it may not be necessary to address both at the same time. I'd be very curious to see if there's an example of a situation where it's unavoidable that unrecognized property warnings get logged by Kafka Streams, even when fully leveraging the namespacing mechanisms provided by the {{{}producer.{}}}, {{{}main.consumer.{}}}, etc. prefixes. I've updated [https://github.com/apache/kafka/pull/11986] to remove any changes to the public Java clients API, so it's ready for review if anyone wants to take a look. > 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: Chris Egerton >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 (v8.20.1#820001)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17519199#comment-17519199 ] Matthias J. Sax commented on KAFKA-7509: Thanks for staying on top of this. The filtering code is a partial solution to the problem. Also the prefixes like `producer.` et al: yes they help. Not sure if they solve all issues. KAFKA-6793 actually illustrates the general problem we tried to solve. However, IIRC KAFKA-6793 in particular was triggered by `StreamsConfig#retries` that we needed to pass though the admin client into `StreamsPartitionAssignor`. `StreamConfig#retries` was deprecated (it's still in the code, but unused and not set on the `AdiminConfig` any longer) via KIP-572 [https://cwiki.apache.org/confluence/display/KAFKA/KIP-572%3A+Improve+timeouts+and+retries+in+Kafka+Streams|https://cwiki.apache.org/confluence/display/KAFKA/KIP-572%3A+Improve+timeouts+and+retries+in+Kafka+Streams(] -- thus, KAFKA-6793 might effectively resolved as a side effect, but only because we stopped passing in an unrecognized config – thus, the underlying principle issue is still there. > 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: Chris Egerton >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 (v8.20.1#820001)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17517159#comment-17517159 ] Chris Egerton commented on KAFKA-7509: -- Hmm... I'm not able to reproduce this easily with Kafka Streams. It looks like care is already taken to filter out most irrelevant client properties with logic like [this|https://github.com/apache/kafka/blob/74909e000aaab1f0300ae2e918a6fa361e38078e/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java#L1580-L1598]. I think it might still be possible to trigger the unrecognized config warning by passing in interceptor properties in the top level streams config, but since you can already isolate client properties with separate {{{}producer.{}}}, {{{}consumer.{}}}, and {{admin.}} namespaces, there doesn't seem to be any need to make code changes to address that case. [~mjsax] [~vvcephei] do either of you know if KAFKA-6793 is still an issue? And if so, do you know off the top of your head how to go about reproducing these warning messages with Kafka Streams? If Kafka Streams has already taken care of everything then that should simplify the review process for [https://github.com/apache/kafka/pull/11986]. > 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: Chris Egerton >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 (v8.20.1#820001)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17516992#comment-17516992 ] Chris Egerton commented on KAFKA-7509: -- [~mjsax] thanks for taking a look. RE {{connect.kafka.cluster.id}} – this property is already passed in automatically by [ConnectUtils::addMetricsContextProperties|https://github.com/apache/kafka/blob/74909e000aaab1f0300ae2e918a6fa361e38078e/connect/runtime/src/main/java/org/apache/kafka/connect/util/ConnectUtils.java#L144-L153], which is used all over the place in Connect (see [here|https://github.com/apache/kafka/blob/74909e000aaab1f0300ae2e918a6fa361e38078e/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java#L664], [here|https://github.com/apache/kafka/blob/74909e000aaab1f0300ae2e918a6fa361e38078e/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java#L697], and [here|https://github.com/apache/kafka/blob/74909e000aaab1f0300ae2e918a6fa361e38078e/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java#L742] for some cases). The changes in the PR don't add any new config properties, they just cause the ones injected by that method to be ignored when logging unused properties. I can try to take a stab at this with Streams, will probably file a separate PR for that if it looks promising. > 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: Chris Egerton >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 (v8.20.1#820001)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17516979#comment-17516979 ] Matthias J. Sax commented on KAFKA-7509: Thanks for following up. – I briefly took a look. My first impression is that it might work with KS. Assing `connect.kafka.cluster.id` would still require a KIP – it seems that adding a config and avoiding the warnings is two different things though – might be good to extract to changes for the logging into an independent PR (and maybe add using is in KS right away to verify that it works)? > 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: Chris Egerton >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 (v8.20.1#820001)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17516226#comment-17516226 ] Chris Egerton commented on KAFKA-7509: -- Alright, I dug a little deeper and it looks like this has been discussed in even greater detail in KAFKA-6793 and [KIP-552|https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=142641934]. The progress there stalled after the concept of the {{RecordingMap}} mechanism was brought up. I think that that approach has some merit and have put together a proof of concept here: [https://github.com/apache/kafka/pull/11986] This may still require a KIP (possibly just alterations to the existing KIP-552), but it's borderline. [~mjsax] does it look like this same approach might work with Streams too? > 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: Chris Egerton >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 (v8.20.1#820001)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17510997#comment-17510997 ] Matthias J. Sax commented on KAFKA-7509: Thanks for the detailed explanation. I don't know anything about connect code base, and just realizing it's even more complicated than for the Streams case with regard to nesting... I guess I don't have a good proposal how to address it. – In the end, it also seems a bigger problem for Connect than for Streams (for Streams, it falls into "annoying but not harmful" category, but it seem for Connect the noise ration is much higher...). > 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 >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 (v8.20.1#820001)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17509303#comment-17509303 ] Chris Egerton commented on KAFKA-7509: -- RE backwards compatibility: That's a fair point about feature flag gating. You could have that gating at multiple levels in Connect: # Gating for the top-level worker config (either on an all-or-nothing basis, with one property per case, or something else entirely), which would affect these cases that are currently configured using the entire worker config: ## The Kafka clients that the worker creates for managing internal topics, fetching the Kafka cluster ID, etc. (possibly even one property for each type of client–admin, producer, and consumer) ## REST extensions brought up on the worker ## The connector client config override policy used by the worker ## Admin clients that are brought up on behalf of connectors (see KAFKA-9046 for more context on this special case) # Gating for interceptors used by each Kafka client that's configured in the worker config ## The Kafka clients used for internal topics, etc. described in section 1.1 ## The admin clients brought up on behalf of connectors described in section 1.4 ## The Kafka clients used for connectors (configured using the {{{}consumer.{}}}, {{{}producer.{}}}, and {{admin.}} prefixes) # Gating for interceptors used by each Kafka client that's configured in a connector config ## There's just one case for this (configured using the {{{}consumer.override.{}}}, {{{}producer.override.{}}}, and {{admin.override.}} prefixes) We could also theoretically introduce namespacing for connector configs so that {{Connector}} instances don't receive the {{{}key.converter.*{}}}, {{{}value.converter.*{}}}, {{{}header.converter.*{}}}, {{{}transforms.*{}}}, and {{predicates.*}} config properties, although given the currently-mostly-benign KAFKA-9228, we might want to hold off on that. It would allow {{Connector}} (and probably {{{}Task{}}}) implementations to log warnings on unrecognized config properties though, which they can't realistically do right now without producing a ton of false positives. At the Kafka client level, this approach seems pretty simple, but given all of the nesting that can happen in Kafka Connect with fairly standard usage, it does get more complicated. I think a reasonable way to make this happen could be to use the same config property in the Connect worker config that we would in Kafka clients–something like {{config.namespacing.enabled}} (name obviously subject to change), which could be specified in the worker config at the top level and would control both namespacing behavior for all 1.* cases outlined above, and establish a default for all 2.* and 3.* cases, which could then be overridden using existing mechanisms in Connect for overriding worker defaults. For example, to disable namespacing by default for all connector producers on your worker, you could specify {{{}producer.config.namespacing.enabled = false{}}}. RE encapsulation: There's a few cases where it's been useful for Connect worker plugins to be aware of the worker config that I've encountered: * REST extensions: ** Monitoring the internal topics of the Connect worker in order to expose extra metadata about connector health, like injecting a "last modified" time into the response from the {{GET /connector/\{connector}/status}} endpoint ** Modifying key/value/header converter configurations before a connector is created, which requires knowledge of the {{key.converter.* / }}{{value.converter.* }}/ {{header.converter.*}} properties in the worker config in order to handle the case that the connector config doesn't contain overrides for the given converter type * Config providers (which don't receive the full worker config at the moment, but could possibly benefit from receiving it): ** Store and retrieve secrets using a Kafka topic on the same cluster that the Connect worker uses for its internal topics I also wonder if a consumer interceptor might benefit from knowing how to access the Kafka cluster that the consumer is reading from in order to write custom metrics to a Kafka topic, although I haven't personally done too much work in this space so I don't know for sure if it's a realistic use case. All of these cases can be easily addressed by requiring users to duplicate top-level configs into namespaced configs (e.g., specify the {{{}bootstrap.servers{}}}, {{{}sasl.jaas.config{}}}, {{{}security.protocol{}}}, etc. properties twice in your worker config: once at the top level, and once with something like a {{rest.extension.foo.*}} namespace. But that'd be awful for end users who will inevitably ask why we couldn't come up with something a little more sophisticated in order to save them work copy+pasting. Introducing a {{forward.}} prefix might help but the more I think about
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17509095#comment-17509095 ] Matthias J. Sax commented on KAFKA-7509: {quote} # Breaking backwards compatibility for a lot of Kafka components (producer/consumer interceptors, Connect REST extensions, etc.{quote} Why? Could we not feature flag it (disable by default to preserve backward compatibility; maybe log a WARN message if not enabled; have a path forward to enable by default in some future major release)? {quote} # Complicating the configuration behavior for applications with this type of "nested" configuration surface{quote} Yes, it's a trade-off between flexibility and complexity – personally, I think that for most cases deep nesting won't be the case, and single-level nesting seems not to be too complicated and more flexible compare to a "config enable/disable warn-logging". {quote} # Makes it impossible to have sub-configured components that are aware of their parent's configuration (for example, you could no longer have a Connect REST extension that's aware of the entire Connect worker's config without duplicating those same properties into the sub-config for that REST extension){quote} This is a use-case I never encountered before. From an encapsulation point of view, I am wondering why an inner component would need the config of the outer one? Can you elaborate? {quote}Ultimately, this still seems a little too aggressive of a change for the problem that it's trying to solve. If we were redesigning these APIs from the ground up, it would certainly be beneficial, but considering how much has been built on top of these APIs already and how much work it'd take for users to adjust to the proposed changes, it doesn't seem like a friendly tradeoff. Plus, for some situations (like the example with REST extensions), it's unclear how we'd want to proceed. {quote} Yeah, that always tricky. Personally, I tend to prefer "the right" solution even if it might be more complex to get there. But I don't feel too strong about it either. I agree it's "just" about log messages, so maybe not worth it. My personal concern with disabling (either completely via config or move to DEBUG) is that actual useful warning would be affected, too. It just seems to be too coarse grained – for this case, I was rather have spurious / annoying WARNs than allow to disable them. But I guess details would need to get discussed on a KIP anyway... > 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 >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 (v8.20.1#820001)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17507868#comment-17507868 ] Chris Egerton commented on KAFKA-7509: -- Thanks [~mjsax]--I think that's an acceptable approach, but it would come at a few costs: # Breaking backwards compatibility for a lot of Kafka components (producer/consumer interceptors, Connect REST extensions, etc.) # Complicating the configuration behavior for applications with this type of "nested" configuration surface # Makes it impossible to have sub-configured components that are aware of their parent's configuration (for example, you could no longer have a Connect REST extension that's aware of the entire Connect worker's config without duplicating those same properties into the sub-config for that REST extension) I personally hesitate to inflict a breaking change on users just for the sake of cleaning up log messages, but the second point actually provides a bit of an opportunity to clean up some of the uglier parts of Connect (and possibly other projects as well). Right now, if you want to configure a Connect worker (or connector) with credentials to use for its connectors' Kafka clients, you have to specify the same credentials in up to three different places: one for each type of Kafka client. (This isn't a hypothetical scenario; it's required if you want to run a sink connector that uses the DLQ feature.) The ability to define a single "forward." (or possibly just "clients.") prefix for properties to be passed to all nested configurations (or all nested configurations used exclusively for Kafka clients) would help with situations like this by eliminating the need to duplicate properties within a single configuration. Ultimately, this still seems a little too aggressive of a change for the problem that it's trying to solve. If we were redesigning these APIs from the ground up, it would certainly be beneficial, but considering how much has been built on top of these APIs already and how much work it'd take for users to adjust to the proposed changes, it doesn't seem like a friendly tradeoff. Plus, for some situations (like the example with REST extensions), it's unclear how we'd want to proceed. > 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 >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 (v8.20.1#820001)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17507833#comment-17507833 ] Matthias J. Sax commented on KAFKA-7509: Personally, I think the best way would be to tackle this with a KIP across the board: not just for connect. The idea would be to allow user to add a prefix to the config they want to "forward" to "inner" classes. For example, if an interceptor needs a config, we could use `interceptor.foo.bar` instead of `foo.bar` – this way, we know that the consumer/producer can ignore this config, and when we forward the config into the interceptor, we actually remove `interceptor` prefix. The `interceptor` prefix would of course be quite specific (and it only works because interceptors are a Kafka concept). We could also generalize it using a prefix `forward.`. In Kafka Streams, we are already using `main.consumer.`, `restore.consumer.`, and `producer.` as prefix to allow users to configure individual client and to avoid that `StreamsConfig` logs warnings about "unknown" client config. The advantage of the prefix approach is, that is it much more fine-grained compared to a config, and it can also be applies recursively. For example, in Kafka Streams we use an `AdminClient` inside the `ConsumerClient`. If we want to configure the admin client, we can provide `main.consumer.forward.actualAdminConfigName` and avoid that the consumer logs a warning about an unknown client. > 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 >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 (v8.20.1#820001)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17507768#comment-17507768 ] Chris Egerton commented on KAFKA-7509: -- Bumping this again as the problem continues to persist for Connect (and possibly other projects). While I agree that there's value in logging warnings about unrecognized values in some contexts, that value is lost when the false positive rate for these warnings becomes too high, which appears to be the case for Connect. As [~rhauch] has noted with his exhaustive research into this problem space, it's basically impossible at the moment for Connect to preemptively filter out irrelevant values for some of the Kafka clients that it creates. And, as [~mjsax] has noted, it's not an option to unconditionally demote these log messages to {{DEBUG}} level for all Kafka clients, since there are plenty of projects that do not generate these kinds of false positives, in which case these messages do serve as effective warnings that there may be a typo in the client's config. One compromise could be to do the following: # Add a new {{AbstractConfig::logUnused}} variant that operates at {{DEBUG}} level instead of {{WARN}} (exact details negotiable; this could be an overloading of the existing {{logUnused}} method that accepts a boolean or some other parameter to toggle the log level used, or a separate {{debugUnused}} method that's hardcoded to use {{{}DEBUG{}}}-level logging, or something else entirely). # Add a new {{warn.on.unused}} boolean property to the producer, consumer, and admin client API, with a default of {{{}true{}}}. When enabled, the current behavior of logging unrecognized values at {{WARN}} level will take place. When disabled (i.e., set to {{{}false{}}}), unrecognized values will be instead logged at {{DEBUG}} level, using the new {{AbstractConfig}} API from part 1. # Modify Connect to set {{warn.on.unused}} to {{false}} by default for all Kafka clients that it uses for its internal topics (and possibly in other contexts where the rate of false positives is also high). Allow users to override this default by explicitly specifying a value for the property in the Connect worker config. This would be completely backwards compatible outside of Connect but also allow for other projects to opt in to a reduction in {{{}WARN{}}}-level log spam, and within Connect, would greatly reduce the false positive rate for {{{}WARN{}}}-level messages. Obviously, it would require a KIP, but I'm happy to write one if this seems like a reasonable high-level approach. Thoughts? > 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 (v8.20.1#820001)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17063667#comment-17063667 ] ASF GitHub Bot commented on KAFKA-7509: --- rhauch commented on pull request #5876: KAFKA-7509: Avoid passing most 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 to remove the any properties for the producer 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 be used by both 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. Connect configures the serdes for the producers, so we're left with any custom properties for interceptors and partitioners still getting passed to the AdminClient, but 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 and minimize changes to other existing code. 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). Also, note that when configuring the producers and consumers for connectors, Connect only uses the properties that begin with the `producer.` and `consumer.` prefixes, respectively. 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 to 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
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17030842#comment-17030842 ] ASF GitHub Bot commented on KAFKA-7509: --- rhauch commented on pull request #5876: KAFKA-7509: Avoid passing most non-applicable properties to producer, consumer, and admin client URL: https://github.com/apache/kafka/pull/5876 This is an automated message from the Apache Git Service. To respond to the message, please log on to 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 (v8.3.4#803005)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17030839#comment-17030839 ] Randall Hauch commented on KAFKA-7509: -- KAFKA-6793 and [https://github.com/apache/kafka/pull/8043|https://github.com/apache/kafka/pull/8043/files] appears to be a simple fix that will eliminate these warnings (by making them debug statements). Much more straightforward than this proposal, so closing this as WONTFIX and closing the [https://github.com/apache/kafka/pull/5876] PR without merging. > 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 (v8.3.4#803005)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16871773#comment-16871773 ] Patrik Kleindl commented on KAFKA-7509: --- Not related to Connect but giving this a bump because of [https://confluentcommunity.slack.com/archives/C48AHTCUQ/p1561063563237400], maybe it will help get this moving again. The warning happens in Kafka Streams on several occasions: * admin.retries because this is passed from streams itself to the admin client * when trying to override a property for e.g. main.consumer.max.poll.records * when trying to pass a property through the application properties to the RocksDBConfigSetter (they are added automatically there, but also produce warnings) The first two pop up because the prefixes are stripped from the properties when they are used but not handled in the comparison. And the code does not filter out properties based on the prefix if they are not even aimed at the receiver (e.g. admin.retries should not affect the producer). The third would qualify for the "passthrough" approach mentioned above. > 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)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16673834#comment-16673834 ] Randall Hauch commented on KAFKA-7509: -- Since changing the unused property log messages to debug was not the correct change, I closed that PR and took yet another approach that just filters out worker properties before passing them to the producer, consumer, and admin clients. 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|https://github.com/apache/kafka/pull/5876], Connect now 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 still does not go as far as I'd like. It'd be ideal if Connect could use a RecordingMap to know which properties were used across _all_ of the clients and the worker itself, and to then log _only those_ properties that were never used at all. This is beyond the scope of this issue, however, and can be addressed in a follow-on issue that will require changes to AbstractConfig. > 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)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16673828#comment-16673828 ] ASF GitHub Bot commented on KAFKA-7509: --- rhauch closed pull request #5867: KAFKA-7509: Logging unused configs as DEBUG rather than WARN URL: https://github.com/apache/kafka/pull/5867 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index 9cf13dd35df..82dbb8a7153 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -284,7 +284,7 @@ private void logAll() { */ public void logUnused() { for (String key : unused()) -log.warn("The configuration '{}' was supplied but isn't a known config.", key); +log.debug("The configuration '{}' was supplied but isn't a known config.", key); } /** 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)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16673371#comment-16673371 ] chris riley commented on KAFKA-7509: Just commenting to say these log warnings have been a thorn in my side for some time now. I was about to create a PR to do the same thing as [~rhauch] did. As a workaround I have set loglevel to ERROR in my logback for {noformat} org.apache.kafka.clients.admin.AdminClientConfig org.apache.kafka.clients.consumer.ConsumerConfig org.apache.kafka.clients.producer.ProducerConfig{noformat} But I would rather not have to do this. > 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)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16672356#comment-16672356 ] Matthias J. Sax commented on KAFKA-7509: While logging in DEBUG level would prevent miss leading WARN log entries, it seems to defeat the purpose, that is to guard against typing mistakes in properties files etc. (ie, when plain strings instead of predefined const values are used). Thus, I am not 100% convinced of this solution. Thinking about this, we could instead define a prefix, that is use for all configs that should be passed through. For all configs with this prefix, no name check is done, and the prefix is stripped before it's passed into the internal call to `configure()`. For example, Kafka Streams would not set `admin.retries`, but `passthrough.admin.retries` avoiding the WARN log. (I hope we find a better name than "passthrough" :) ) This would be a public API change though and require a KIP. Thoughts? > 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)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16672097#comment-16672097 ] Randall Hauch commented on KAFKA-7509: -- The most recent [PR #5867|https://github.com/apache/kafka/pull/5867] would also fix KAFKA-6793 for Streams. > 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)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16672089#comment-16672089 ] ASF GitHub Bot commented on KAFKA-7509: --- rhauch opened a new pull request #5867: KAFKA-7509: Logging unused configs as DEBUG rather than WARN URL: https://github.com/apache/kafka/pull/5867 The KafkaProducer, KafkaConsumer, and KafkaAdminClient all call `config.logUnused()` to log all of the unused configurations. These can be very misleading, because some conventional configuration properties for custom interceptors, key or value (de)serializers, metrics reporters, and partitioner are not actually defined by ConfigDefs and are thus all considered unused and logged as warnings. Also, some client applications (such as Connect) do not determine the subset of properties that can be passed to one of these clients, since there are these custom extensions. This causes extra properties to be logged as warnings, which is concerning for new users. These unused configurations should be logged at DEBUG level instead to reduce the number of unexpected warnings in many client application logs. ### Backporting This is a usability improvement that affects only this particular log message. As such, it may or may not be something that should be backported. Advice from committers would be appreciated. ### 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: 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)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16672080#comment-16672080 ] ASF GitHub Bot commented on KAFKA-7509: --- rhauch closed pull request #5802: KAFKA-7509: Reduce unnecessary and misleading “configuration supplied but not known” warning messages in Connect URL: https://github.com/apache/kafka/pull/5802 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java b/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java index 058c491672a..ebd10d45689 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java @@ -185,6 +185,20 @@ protected AdminClientConfig(Map props, boolean doLog) { return CONFIG.names(); } +/** + * Return whether the given property name is a known configuration. This will consider valid any property that can be passed to + * instances of extensions, such as the {@link #METRIC_REPORTER_CLASSES_CONFIG metrics reporter}. + * + * @param name the property name + * @return true if the supplied name matches a known property, or false if it is unknown + */ +public static boolean isKnownConfig(String name) { +if (name == null) { +return false; +} +return configNames().contains(name) || name.startsWith(METRIC_REPORTER_CLASSES_CONFIG); +} + public static void main(String[] args) { System.out.println(CONFIG.toHtmlTable()); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index ddd6e06c713..c62c574ad80 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -507,6 +507,22 @@ protected ConsumerConfig(Map props, boolean doLog) { return CONFIG.names(); } +/** + * Return whether the given property name is a known configuration. This will consider valid any property that can be passed to + * instances of extensions, such as the {@link #METRIC_REPORTER_CLASSES_CONFIG metrics reporter}. + * + * @param name the property name + * @return true if the supplied name matches a known property, or false if it is unknown + */ +public static boolean isKnownConfig(String name) { +if (name == null) { +return false; +} +return configNames().contains(name) + || name.startsWith(KEY_DESERIALIZER_CLASS_CONFIG) || name.startsWith(VALUE_DESERIALIZER_CLASS_CONFIG) + || name.startsWith(METRIC_REPORTER_CLASSES_CONFIG) || name.startsWith(INTERCEPTOR_CLASSES_CONFIG); +} + public static void main(String[] args) { System.out.println(CONFIG.toHtmlTable()); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 6142519c4dc..515a54888e2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -392,6 +392,23 @@ public ProducerConfig(Map props) { return CONFIG.names(); } +/** + * Return whether the given property name is a known configuration. This will consider valid any property that can be passed to + * instances of extensions, such as the {@link #METRIC_REPORTER_CLASSES_CONFIG metrics reporter}. + * + * @param name the property name + * @return true if the supplied name matches a known property, or false if it is unknown + */ +public static boolean isKnownConfig(String name) { +if (name == null) { +return false; +} +return configNames().contains(name) + || name.startsWith(KEY_SERIALIZER_CLASS_CONFIG) || name.startsWith(VALUE_SERIALIZER_CLASS_CONFIG) + || name.startsWith(METRIC_REPORTER_CLASSES_CONFIG) || name.startsWith(INTERCEPTOR_CLASSES_CONFIG) + || name.startsWith(PARTITIONER_CLASS_CONFIG); +} + public static void main(String[] args) { System.out.println(CONFIG.toHtmlTable()); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index df73a434d31..85a8d320d0c 100644 ---
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16672075#comment-16672075 ] Randall Hauch commented on KAFKA-7509: -- Okay, I am now to the point where I think this "hygiene" approach is completely flawed, and that instead the problem is that we're logging as warnings all "unused" properties in the producer, consumer, and the KafkaAdminClient. Let me back up. Based upon the discussion above, I modified my approach to attempt to retain all of the configuration properties that are known to the ProducerConfig, ConsumerConfig, and AdminClientConfig, where "known" properties include: * all of those whose name is in the set returned by each of the config's configNames(), or * any property that can be passed to an interceptor, key or value (de)serializer, metrics reporter, or partitioner that is instantiated by the client. That last bit is the problem: the properties to the clients' interceptors, serdes, metrics reporter, and partitions are all unprefixed, so it is impossible to know which properties are needed by any of the specified implementations. IOW, the properties passed to a producer, consumer, or admin client must be able to include any property that is needed by any of these custom components. And, because the getConfiguredComponent method used by the clients passes the Map to the component's configure method, the AbstractConfig doesn't know whether those properties are even used by the component. So, if the AbstractConfig doesn't really even know whether it a property is really used or unused, why are the Producer, Consumer, and KafkaAdminClient even bothering to log "unused" properties? h3. Bottom line I now posit that the only way to accurately eliminate these warnings is to remove the config.logUnused() call from the Producer, Consumer, and KafkaAdminClient, or to change AbstractConfig.logUnused() to log these at the INFO (or DEBUG) level. > 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: 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)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16660868#comment-16660868 ] Sönke Liebau commented on KAFKA-7509: - Hi [~rhauch], wouldn't this take away the ability to pass configuration options to custom interceptor and partitioner classes? This is probably not something that is widely done, but I can think of a few use cases like making a call to an external system from a partitioner and setting the target url in configuration. Or, come to think of it, the control center interceptors also allow some configuration parameters to be passed to them that are not defined in ProducerConfig I think. > 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: 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)
[jira] [Commented] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16650896#comment-16650896 ] Randall Hauch commented on KAFKA-7509: -- Created a patch with a proposed approach to prevent these warning messages: https://github.com/apache/kafka/pull/5802 > 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: 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)