Skip to content
Closed
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
// Each running query should use its own group id. Otherwise, the query may be only assigned
// partial data since Kafka will assign partitions to multiple consumers having the same group
// id. Hence, we should generate a unique id for each query.
val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}"
val uniqueGroupId = streamingUniqueGroupId(parameters, metadataPath)

val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) }
val specifiedKafkaParams =
Expand Down Expand Up @@ -119,7 +119,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
// Each running query should use its own group id. Otherwise, the query may be only assigned
// partial data since Kafka will assign partitions to multiple consumers having the same group
// id. Hence, we should generate a unique id for each query.
val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}"
val uniqueGroupId = streamingUniqueGroupId(parameters, metadataPath)

val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) }
val specifiedKafkaParams =
Expand Down Expand Up @@ -159,7 +159,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
// Each running query should use its own group id. Otherwise, the query may be only assigned
// partial data since Kafka will assign partitions to multiple consumers having the same group
// id. Hence, we should generate a unique id for each query.
val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}"
val uniqueGroupId = streamingUniqueGroupId(parameters, metadataPath)

val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) }
val specifiedKafkaParams =
Expand Down Expand Up @@ -538,6 +538,17 @@ private[kafka010] object KafkaSourceProvider extends Logging {
.setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer)
.build()

/**
* Returns a unique consumer group (group.id), allowing the user to set the prefix of
* the consumer group
*/
private def streamingUniqueGroupId(parameters: Map[String, String],
metadataPath: String): String = {
val groupIdPrefix = parameters
.getOrElse("group.id.prefix", "spark-kafka-source")
s"${groupIdPrefix}-${UUID.randomUUID}-${metadataPath.hashCode}"
}

/** Class to conveniently update Kafka config params, while logging the changes */
private case class ConfigUpdater(module: String, kafkaParams: Map[String, String]) {
private val map = new ju.HashMap[String, Object](kafkaParams.asJava)
Expand Down