in core/src/main/scala/org/apache/pekko/kafka/ConsumerSettings.scala [273:607]
def withBootstrapServers(bootstrapServers: String): ConsumerSettings[K, V] =
withProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers)
/**
* An id string to pass to the server when making requests. The purpose of this is to be able to track the source
* of requests beyond just ip/port by allowing a logical application name to be included in server-side request logging.
*/
def withClientId(clientId: String): ConsumerSettings[K, V] =
withProperty(ConsumerConfig.CLIENT_ID_CONFIG, clientId)
/**
* A unique string that identifies the consumer group this consumer belongs to.
*/
def withGroupId(groupId: String): ConsumerSettings[K, V] =
withProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
/**
* An id string that marks consumer as a unique static member of the consumer group.
*/
def withGroupInstanceId(groupInstanceId: String): ConsumerSettings[K, V] =
withProperty(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId)
/**
* Scala API:
* The raw properties of the kafka-clients driver, see constants in
* [[org.apache.kafka.clients.consumer.ConsumerConfig]].
*/
def withProperties(properties: Map[String, String]): ConsumerSettings[K, V] =
copy(properties = this.properties ++ properties)
/**
* Scala API:
* The raw properties of the kafka-clients driver, see constants in
* [[org.apache.kafka.clients.consumer.ConsumerConfig]].
*/
def withProperties(properties: (String, String)*): ConsumerSettings[K, V] =
copy(properties = this.properties ++ properties.toMap)
/**
* Java API:
* The raw properties of the kafka-clients driver, see constants in
* [[org.apache.kafka.clients.consumer.ConsumerConfig]].
*/
def withProperties(properties: java.util.Map[String, String]): ConsumerSettings[K, V] =
copy(properties = this.properties ++ properties.asScala)
/**
* The raw properties of the kafka-clients driver, see constants in
* [[org.apache.kafka.clients.consumer.ConsumerConfig]].
*/
def withProperty(key: String, value: String): ConsumerSettings[K, V] =
copy(properties = properties.updated(key, value))
/**
* Java API: Get a raw property. `null` if it is not defined.
*/
def getProperty(key: String): String = properties.getOrElse(key, null)
/**
* Set the maximum duration a poll to the Kafka broker is allowed to take.
*/
def withPollTimeout(pollTimeout: FiniteDuration): ConsumerSettings[K, V] =
copy(pollTimeout = pollTimeout)
/**
* Java API:
* Set the maximum duration a poll to the Kafka broker is allowed to take.
*/
def withPollTimeout(pollTimeout: java.time.Duration): ConsumerSettings[K, V] =
copy(pollTimeout = pollTimeout.asScala)
/**
* Set the interval from one scheduled poll to the next.
*/
def withPollInterval(pollInterval: FiniteDuration): ConsumerSettings[K, V] =
copy(pollInterval = pollInterval)
/**
* Java API:
* Set the interval from one scheduled poll to the next.
*/
def withPollInterval(pollInterval: java.time.Duration): ConsumerSettings[K, V] =
copy(pollInterval = pollInterval.asScala)
/**
* The stage will await outstanding offset commit requests before
* shutting down, but if that takes longer than this timeout it will
* stop forcefully.
*/
def withStopTimeout(stopTimeout: FiniteDuration): ConsumerSettings[K, V] =
copy(stopTimeout = stopTimeout)
/**
* Java API:
* The stage will await outstanding offset commit requests before
* shutting down, but if that takes longer than this timeout it will
* stop forcefully.
*/
def withStopTimeout(stopTimeout: java.time.Duration): ConsumerSettings[K, V] =
copy(stopTimeout = stopTimeout.asScala)
/**
* Set duration to wait for `KafkaConsumer.close` to finish.
*/
def withCloseTimeout(closeTimeout: FiniteDuration): ConsumerSettings[K, V] =
copy(closeTimeout = closeTimeout)
/**
* Java API:
* Set duration to wait for `KafkaConsumer.close` to finish.
*/
def withCloseTimeout(closeTimeout: java.time.Duration): ConsumerSettings[K, V] =
copy(closeTimeout = closeTimeout.asScala)
/**
* If offset commit requests are not completed within this timeout
* the returned Future is completed with [[pekko.kafka.CommitTimeoutException]].
*/
def withCommitTimeout(commitTimeout: FiniteDuration): ConsumerSettings[K, V] =
copy(commitTimeout = commitTimeout)
/**
* Java API:
* If offset commit requests are not completed within this timeout
* the returned Future is completed with [[pekko.kafka.CommitTimeoutException]].
*/
def withCommitTimeout(commitTimeout: java.time.Duration): ConsumerSettings[K, V] =
copy(commitTimeout = commitTimeout.asScala)
/**
* If commits take longer than this time a warning is logged
*/
def withCommitWarning(commitTimeWarning: FiniteDuration): ConsumerSettings[K, V] =
copy(commitTimeWarning = commitTimeWarning)
/**
* Java API:
* If commits take longer than this time a warning is logged
*/
def withCommitWarning(commitTimeWarning: java.time.Duration): ConsumerSettings[K, V] =
copy(commitTimeWarning = commitTimeWarning.asScala)
/**
* Fully qualified config path which holds the dispatcher configuration
* to be used by the [[pekko.kafka.KafkaConsumerActor]]. Some blocking may occur.
*/
def withDispatcher(dispatcher: String): ConsumerSettings[K, V] =
copy(dispatcher = dispatcher)
/**
* If set to a finite duration, the consumer will re-send the last committed offsets periodically
* for all assigned partitions.
*
* @see https://issues.apache.org/jira/browse/KAFKA-4682
*/
def withCommitRefreshInterval(commitRefreshInterval: Duration): ConsumerSettings[K, V] =
copy(commitRefreshInterval = commitRefreshInterval)
/**
* Java API:
* If set to a finite duration, the consumer will re-send the last committed offsets periodically
* for all assigned partitions. @see https://issues.apache.org/jira/browse/KAFKA-4682
* Set to [[java.time.Duration.ZERO]] to switch it off.
*
* @see https://issues.apache.org/jira/browse/KAFKA-4682
*/
def withCommitRefreshInterval(commitRefreshInterval: java.time.Duration): ConsumerSettings[K, V] =
if (commitRefreshInterval.isZero) copy(commitRefreshInterval = Duration.Inf)
else copy(commitRefreshInterval = commitRefreshInterval.asScala)
/**
* Time to wait for pending requests when a partition is closed.
*/
def withWaitClosePartition(waitClosePartition: FiniteDuration): ConsumerSettings[K, V] =
copy(waitClosePartition = waitClosePartition)
/**
* Enable kafka connection checker with provided settings
*/
def withConnectionChecker(
kafkaConnectionCheckerConfig: ConnectionCheckerSettings): ConsumerSettings[K, V] =
copy(connectionCheckerConfig = kafkaConnectionCheckerConfig)
/**
* Java API:
* Time to wait for pending requests when a partition is closed.
*/
def withWaitClosePartition(waitClosePartition: java.time.Duration): ConsumerSettings[K, V] =
copy(waitClosePartition = waitClosePartition.asScala)
/** Scala API: Limits the blocking on Kafka consumer position calls. */
def withPositionTimeout(positionTimeout: FiniteDuration): ConsumerSettings[K, V] =
copy(positionTimeout = positionTimeout)
/** Java API: Limits the blocking on Kafka consumer position calls. */
def withPositionTimeout(positionTimeout: java.time.Duration): ConsumerSettings[K, V] =
copy(positionTimeout = positionTimeout.asScala)
/** Scala API: Limits the blocking on Kafka consumer offsetForTimes calls. */
def withOffsetForTimesTimeout(offsetForTimesTimeout: FiniteDuration): ConsumerSettings[K, V] =
copy(offsetForTimesTimeout = offsetForTimesTimeout)
/** Java API: Limits the blocking on Kafka consumer offsetForTimes calls. */
def withOffsetForTimesTimeout(offsetForTimesTimeout: java.time.Duration): ConsumerSettings[K, V] =
copy(offsetForTimesTimeout = offsetForTimesTimeout.asScala)
/** Scala API */
def withMetadataRequestTimeout(metadataRequestTimeout: FiniteDuration): ConsumerSettings[K, V] =
copy(metadataRequestTimeout = metadataRequestTimeout)
/** Java API */
def withMetadataRequestTimeout(metadataRequestTimeout: java.time.Duration): ConsumerSettings[K, V] =
copy(metadataRequestTimeout = metadataRequestTimeout.asScala)
/** Scala API: Check interval for TransactionalProducer when finishing transaction before shutting down consumer */
def withDrainingCheckInterval(drainingCheckInterval: FiniteDuration): ConsumerSettings[K, V] =
copy(drainingCheckInterval = drainingCheckInterval)
/** Java API: Check interval for TransactionalProducer when finishing transaction before shutting down consumer */
def withDrainingCheckInterval(drainingCheckInterval: java.time.Duration): ConsumerSettings[K, V] =
copy(drainingCheckInterval = drainingCheckInterval.asScala)
/** Scala API */
def withPartitionHandlerWarning(partitionHandlerWarning: FiniteDuration): ConsumerSettings[K, V] =
copy(partitionHandlerWarning = partitionHandlerWarning)
/** Java API */
def withPartitionHandlerWarning(partitionHandlerWarning: java.time.Duration): ConsumerSettings[K, V] =
copy(partitionHandlerWarning = partitionHandlerWarning.asScala)
/**
* Scala API.
* A hook to allow for resolving some settings asynchronously.
* @since Alpakka Kafka 2.0.0
*/
def withEnrichAsync(value: ConsumerSettings[K, V] => Future[ConsumerSettings[K, V]]): ConsumerSettings[K, V] =
copy(enrichAsync = Some(value))
/**
* Java API.
* A hook to allow for resolving some settings asynchronously.
* @since Alpakka Kafka 2.0.0
*/
def withEnrichCompletionStage(
value: java.util.function.Function[ConsumerSettings[K, V], CompletionStage[ConsumerSettings[K, V]]])
: ConsumerSettings[K, V] =
copy(enrichAsync = Some((s: ConsumerSettings[K, V]) => value.apply(s).asScala))
/**
* Replaces the default Kafka consumer creation logic.
*/
def withConsumerFactory(
factory: ConsumerSettings[K, V] => Consumer[K, V]): ConsumerSettings[K, V] = copy(consumerFactory = factory)
/**
* Set the protection for unintentional offset reset.
*/
def withResetProtectionSettings(resetProtection: OffsetResetProtectionSettings): ConsumerSettings[K, V] =
copy(resetProtectionSettings = resetProtection)
/**
* Get the Kafka consumer settings as map.
*/
def getProperties: java.util.Map[String, AnyRef] = properties.asInstanceOf[Map[String, AnyRef]].asJava
def getCloseTimeout: java.time.Duration = closeTimeout.asJava
def getPositionTimeout: java.time.Duration = positionTimeout.asJava
def getOffsetForTimesTimeout: java.time.Duration = offsetForTimesTimeout.asJava
def getMetadataRequestTimeout: java.time.Duration = metadataRequestTimeout.asJava
private def copy(
properties: Map[String, String] = properties,
keyDeserializer: Option[Deserializer[K]] = keyDeserializerOpt,
valueDeserializer: Option[Deserializer[V]] = valueDeserializerOpt,
pollInterval: FiniteDuration = pollInterval,
pollTimeout: FiniteDuration = pollTimeout,
stopTimeout: FiniteDuration = stopTimeout,
closeTimeout: FiniteDuration = closeTimeout,
commitTimeout: FiniteDuration = commitTimeout,
commitTimeWarning: FiniteDuration = commitTimeWarning,
commitRefreshInterval: Duration = commitRefreshInterval,
dispatcher: String = dispatcher,
waitClosePartition: FiniteDuration = waitClosePartition,
positionTimeout: FiniteDuration = positionTimeout,
offsetForTimesTimeout: FiniteDuration = offsetForTimesTimeout,
metadataRequestTimeout: FiniteDuration = metadataRequestTimeout,
drainingCheckInterval: FiniteDuration = drainingCheckInterval,
enrichAsync: Option[ConsumerSettings[K, V] => Future[ConsumerSettings[K, V]]] = enrichAsync,
consumerFactory: ConsumerSettings[K, V] => Consumer[K, V] = consumerFactory,
connectionCheckerConfig: ConnectionCheckerSettings = connectionCheckerSettings,
partitionHandlerWarning: FiniteDuration = partitionHandlerWarning,
resetProtectionSettings: OffsetResetProtectionSettings = resetProtectionSettings): ConsumerSettings[K, V] =
new ConsumerSettings[K, V](
properties,
keyDeserializer,
valueDeserializer,
pollInterval,
pollTimeout,
stopTimeout,
closeTimeout,
commitTimeout,
commitRefreshInterval,
dispatcher,
commitTimeWarning,
waitClosePartition,
positionTimeout,
offsetForTimesTimeout,
metadataRequestTimeout,
drainingCheckInterval,
enrichAsync,
consumerFactory,
connectionCheckerConfig,
partitionHandlerWarning,
resetProtectionSettings)
/**
* Applies `enrichAsync` to complement these settings from asynchronous sources.
*/
def enriched: Future[ConsumerSettings[K, V]] =
enrichAsync.map(_.apply(this.copy(enrichAsync = None))).getOrElse(Future.successful(this))
/**
* Create a [[org.apache.kafka.clients.consumer.Consumer Kafka Consumer]] instance from these settings.
*
* This will fail with `IllegalStateException` if asynchronous enrichment is set up -- always prefer [[createKafkaConsumerAsync()]] or [[createKafkaConsumerCompletionStage()]].
*
* @throws IllegalStateException if asynchronous enrichment is set via `withEnrichAsync` or `withEnrichCompletionStage`, you must use `createKafkaConsumerAsync` or `createKafkaConsumerCompletionStage` to apply it
*/
def createKafkaConsumer(): Consumer[K, V] =
if (enrichAsync.isDefined) {
throw new IllegalStateException(
"Asynchronous settings enrichment is set via `withEnrichAsync` or `withEnrichCompletionStage`, you must use `createKafkaConsumerAsync` or `createKafkaConsumerCompletionStage` to apply it")
} else {
consumerFactory.apply(this)
}