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

Guozhang Wang commented on KAFKA-6166:
--------------------------------------

[~wushujames] asked about this issue as well, so just copy-pasting my replies 
here:

{code}
Thanks for bringing this up.

The original design is to let users apply the prefix ONLY if they want to 
override for the specific client. For example, for `metric.reporters` or 
`interceptor.classes` which are defined for both producer and consumer (and 
moving forward we are adding admin client as well), without the prefix they 
will be applied to all embedded clients; and if users want to have different 
config value overrides for different clients, they can use the prefix.

So is you define for `metrics.reporter` it should be applied to both producer 
and consumer (is it not the case for you? Please confirm).

For custom config k-v pairs, I think we did not discuss about this in-depth. 
But our currently implementation is, indeed, restricting to producer / consumer 
client prop definition only, for example.

getClientPropsWithPrefix(CONSUMER_PREFIX, ConsumerConfig.configNames());
props.putAll(originalsWithPrefix(prefix));

so if you do not add the prefix to your custom k-v pairs, it will be skipped. 
Thinking about it a bit, I think we should probably do what you were inclining 
to, i.e. to define custom k-v pairs also globally unless overridden by prefix. 
{code}


> Streams configuration requires consumer. and producer. in order to be read
> --------------------------------------------------------------------------
>
>                 Key: KAFKA-6166
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6166
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 0.11.0.0
>         Environment: Kafka 0.11.0.0
> JDK 1.8
> CoreOS
>            Reporter: Justin Manchester
>            Priority: Minor
>
> Problem:
> In previous release you could specify a custom metrics reporter like so:
> Properties config = new Properties(); 
> config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092"); 
> config.put(StreamsConfig.METRIC_REPORTER_CLASSES_CONFIG, 
> "com.mycompany.MetricReporter"); 
> config.put("custom-key-for-metric-reporter", "value");
> From 0.11.0.0 onwards this is no longer possible, as you have to specify 
> consumer.custom-key-for-metric-reporter or 
> producer.custom-key-for-metric-reporter otherwise it's stripped out of the 
> configuration.
> So, if you wish to use a metrics reporter and to collect producer and 
> consumer metrics, as well as kafka-streams metrics, that you would need to 
> specify 3 distinct configs:
> 1) consumer.custom-key-for-metric-reporter 
> 2) producer.custom-key-for-metric-reporter 
> 3) custom-key-for-metric-reporter
> This appears to be a regression.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to