@Override public Map<String, Object> appConfigsWithPrefix(final String prefix) { return config.originalsWithPrefix(prefix); }
private Map<String, Object> getClientPropsWithPrefix(final String prefix, final Set<String> configNames) { final Map<String, Object> props = clientProps(configNames, originals()); props.putAll(originalsWithPrefix(prefix)); return props; }
/** * Get a map of custom configs by removing from the originals all the Streams, Consumer, Producer, and AdminClient configs. * Prefixed properties are also removed because they are already added by {@link #getClientPropsWithPrefix(String, Set)}. * This allows to set a custom property for a specific client alone if specified using a prefix, or for all * when no prefix is used. * * @return a map with the custom properties */ private Map<String, Object> getClientCustomProps() { final Map<String, Object> props = originals(); props.keySet().removeAll(CONFIG.names()); props.keySet().removeAll(ConsumerConfig.configNames()); props.keySet().removeAll(ProducerConfig.configNames()); props.keySet().removeAll(AdminClientConfig.configNames()); props.keySet().removeAll(originalsWithPrefix(CONSUMER_PREFIX, false).keySet()); props.keySet().removeAll(originalsWithPrefix(PRODUCER_PREFIX, false).keySet()); props.keySet().removeAll(originalsWithPrefix(ADMIN_CLIENT_PREFIX, false).keySet()); return props; }
/** * Get the configs for the {@link KafkaConsumer global consumer}. * Properties using the prefix {@link #GLOBAL_CONSUMER_PREFIX} will be used in favor over * the properties prefixed with {@link #CONSUMER_PREFIX} and the non-prefixed versions * (read the override precedence ordering in {@link #GLOBAL_CONSUMER_PREFIX} * except in the case of {@link ConsumerConfig#BOOTSTRAP_SERVERS_CONFIG} where we always use the non-prefixed * version as we only support reading/writing from/to the same Kafka Cluster. * If not specified by {@link #GLOBAL_CONSUMER_PREFIX}, global consumer will share the general consumer configs * prefixed by {@link #CONSUMER_PREFIX}. * * @param clientId clientId * @return Map of the global consumer configuration. */ @SuppressWarnings("WeakerAccess") public Map<String, Object> getGlobalConsumerConfigs(final String clientId) { final Map<String, Object> baseConsumerProps = getCommonConsumerConfigs(); // Get global consumer override configs final Map<String, Object> globalConsumerProps = originalsWithPrefix(GLOBAL_CONSUMER_PREFIX); for (final Map.Entry<String, Object> entry: globalConsumerProps.entrySet()) { baseConsumerProps.put(entry.getKey(), entry.getValue()); } // no need to set group id for a global consumer baseConsumerProps.remove(ConsumerConfig.GROUP_ID_CONFIG); // add client id with stream client id prefix baseConsumerProps.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId + "-global-consumer"); baseConsumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none"); return baseConsumerProps; }
/** * Get the configs for the {@link KafkaConsumer restore-consumer}. * Properties using the prefix {@link #RESTORE_CONSUMER_PREFIX} will be used in favor over * the properties prefixed with {@link #CONSUMER_PREFIX} and the non-prefixed versions * (read the override precedence ordering in {@link #RESTORE_CONSUMER_PREFIX} * except in the case of {@link ConsumerConfig#BOOTSTRAP_SERVERS_CONFIG} where we always use the non-prefixed * version as we only support reading/writing from/to the same Kafka Cluster. * If not specified by {@link #RESTORE_CONSUMER_PREFIX}, restore consumer will share the general consumer configs * prefixed by {@link #CONSUMER_PREFIX}. * * @param clientId clientId * @return Map of the restore consumer configuration. */ @SuppressWarnings("WeakerAccess") public Map<String, Object> getRestoreConsumerConfigs(final String clientId) { final Map<String, Object> baseConsumerProps = getCommonConsumerConfigs(); // Get restore consumer override configs final Map<String, Object> restoreConsumerProps = originalsWithPrefix(RESTORE_CONSUMER_PREFIX); for (final Map.Entry<String, Object> entry: restoreConsumerProps.entrySet()) { baseConsumerProps.put(entry.getKey(), entry.getValue()); } // no need to set group id for a restore consumer baseConsumerProps.remove(ConsumerConfig.GROUP_ID_CONFIG); // add client id with stream client id prefix baseConsumerProps.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId + "-restore-consumer"); baseConsumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none"); return baseConsumerProps; }
public InternalTopicManager(final AdminClient adminClient, final StreamsConfig streamsConfig) { this.adminClient = adminClient; final LogContext logContext = new LogContext(String.format("stream-thread [%s] ", Thread.currentThread().getName())); log = logContext.logger(getClass()); replicationFactor = streamsConfig.getInt(StreamsConfig.REPLICATION_FACTOR_CONFIG).shortValue(); windowChangeLogAdditionalRetention = streamsConfig.getLong(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG); retries = new InternalAdminClientConfig(streamsConfig.getAdminConfigs("dummy")).getInt(AdminClientConfig.RETRIES_CONFIG); log.debug("Configs:" + Utils.NL, "\t{} = {}" + Utils.NL, "\t{} = {}" + Utils.NL, "\t{} = {}", AdminClientConfig.RETRIES_CONFIG, retries, StreamsConfig.REPLICATION_FACTOR_CONFIG, replicationFactor, StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, windowChangeLogAdditionalRetention); for (final Map.Entry<String, Object> entry : streamsConfig.originalsWithPrefix(StreamsConfig.TOPIC_PREFIX).entrySet()) { if (entry.getValue() != null) { defaultTopicConfigs.put(entry.getKey(), entry.getValue().toString()); } } }
final Map<String, Object> mainConsumerProps = originalsWithPrefix(MAIN_CONSUMER_PREFIX); for (final Map.Entry<String, Object> entry: mainConsumerProps.entrySet()) { consumerProps.put(entry.getKey(), entry.getValue()); final Map<String, Object> topicProps = originalsWithPrefix(TOPIC_PREFIX, false);