[ https://issues.apache.org/jira/browse/KAFKA-6166?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16345900#comment-16345900 ]
ASF GitHub Bot commented on KAFKA-6166: --------------------------------------- guozhangwang closed pull request #4489: KAFKA-6166: Update javadoc for streams config URL: https://github.com/apache/kafka/pull/4489 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/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 1393223ec23..ec0a1a8385b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams; +import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -56,9 +57,9 @@ /** * Configuration for a {@link KafkaStreams} instance. - * Can also be used to configure the Kafka Streams internal {@link KafkaConsumer} and {@link KafkaProducer}. - * To avoid consumer/producer property conflicts, you should prefix those properties using - * {@link #consumerPrefix(String)} and {@link #producerPrefix(String)}, respectively. + * Can also be used to configure the Kafka Streams internal {@link KafkaConsumer}, {@link KafkaProducer} and {@link AdminClient}. + * To avoid consumer/producer/admin property conflicts, you should prefix those properties using + * {@link #consumerPrefix(String)}, {@link #producerPrefix(String)} and {@link #adminClientPrefix(String)}, respectively. * <p> * Example: * <pre>{@code @@ -78,8 +79,26 @@ * StreamsConfig streamsConfig = new StreamsConfig(streamsProperties); * }</pre> * + * This instance can also be used to pass in custom configurations to different modules (e.g. passing a special config in your customized serde class). + * The consumer/producer/admin prefix can also be used to distinguish these custom config values passed to different clients with the same config name. + * * Example: + * <pre>{@code + * Properties streamsProperties = new Properties(); + * // sets "my.custom.config" to "foo" for consumer only + * streamsProperties.put(StreamsConfig.consumerPrefix("my.custom.config"), "foo"); + * // sets "my.custom.config" to "bar" for producer only + * streamsProperties.put(StreamsConfig.producerPrefix("my.custom.config"), "bar"); + * // sets "my.custom.config2" to "boom" for all clients universally + * streamsProperties.put("my.custom.config2", "boom"); + * + * // as a result, inside producer's serde class configure(..) function, + * // users can now read both key-value pairs "my.custom.config" -> "foo" + * // and "my.custom.config2" -> "boom" from the config map + * StreamsConfig streamsConfig = new StreamsConfig(streamsProperties); + * }</pre> + * * When increasing both {@link ProducerConfig#RETRIES_CONFIG} and {@link ProducerConfig#MAX_BLOCK_MS_CONFIG} to be more resilient to non-available brokers you should also - * consider increasing {@link ConsumerConfig#MAX_POLL_INTERVAL_MS_CONFIG} using the following guidance: + * consider increasing {@link ConsumerConfig#MAX_POLL_INTERVAL_MS_CONFIG} using the following guidance: * <pre> * max.poll.interval.ms > min ( max.block.ms, (retries +1) * request.timeout.ms ) * </pre> ---------------------------------------------------------------- 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 > 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 > Assignee: Filipe Agapito > Priority: Minor > Labels: newbie++, user-experience > Fix For: 1.1.0, 1.0.1 > > > 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 (v7.6.3#76005)