-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-28163][SS] Use CaseInsensitiveMap for KafkaOffsetReader #24967
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 4 commits
23c2d71
d319e5b
a4790ad
d20b85a
3723930
dcc9d56
5cd7b8e
06fe0fc
9034894
e0e8a88
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -46,7 +46,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap | |
| * properly read. | ||
| */ | ||
| class KafkaContinuousStream( | ||
| offsetReader: KafkaOffsetReader, | ||
| private val offsetReader: KafkaOffsetReader, | ||
|
||
| kafkaParams: ju.Map[String, Object], | ||
| options: CaseInsensitiveStringMap, | ||
| metadataPath: String, | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -56,7 +56,7 @@ import org.apache.spark.util.UninterruptibleThread | |
| * and not use wrong broker addresses. | ||
| */ | ||
| private[kafka010] class KafkaMicroBatchStream( | ||
| kafkaOffsetReader: KafkaOffsetReader, | ||
| private val offsetReader: KafkaOffsetReader, | ||
|
||
| executorKafkaParams: ju.Map[String, Object], | ||
| options: CaseInsensitiveStringMap, | ||
| metadataPath: String, | ||
|
|
@@ -85,7 +85,7 @@ private[kafka010] class KafkaMicroBatchStream( | |
|
|
||
| override def latestOffset(start: Offset): Offset = { | ||
| val startPartitionOffsets = start.asInstanceOf[KafkaSourceOffset].partitionToOffsets | ||
| val latestPartitionOffsets = kafkaOffsetReader.fetchLatestOffsets(Some(startPartitionOffsets)) | ||
| val latestPartitionOffsets = offsetReader.fetchLatestOffsets(Some(startPartitionOffsets)) | ||
| endPartitionOffsets = KafkaSourceOffset(maxOffsetsPerTrigger.map { maxOffsets => | ||
| rateLimit(maxOffsets, startPartitionOffsets, latestPartitionOffsets) | ||
| }.getOrElse { | ||
|
|
@@ -100,7 +100,7 @@ private[kafka010] class KafkaMicroBatchStream( | |
|
|
||
| // Find the new partitions, and get their earliest offsets | ||
| val newPartitions = endPartitionOffsets.keySet.diff(startPartitionOffsets.keySet) | ||
| val newPartitionInitialOffsets = kafkaOffsetReader.fetchEarliestOffsets(newPartitions.toSeq) | ||
| val newPartitionInitialOffsets = offsetReader.fetchEarliestOffsets(newPartitions.toSeq) | ||
| if (newPartitionInitialOffsets.keySet != newPartitions) { | ||
| // We cannot get from offsets for some partitions. It means they got deleted. | ||
| val deletedPartitions = newPartitions.diff(newPartitionInitialOffsets.keySet) | ||
|
|
@@ -117,7 +117,7 @@ private[kafka010] class KafkaMicroBatchStream( | |
| val deletedPartitions = startPartitionOffsets.keySet.diff(endPartitionOffsets.keySet) | ||
| if (deletedPartitions.nonEmpty) { | ||
| val message = | ||
| if (kafkaOffsetReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { | ||
| if (offsetReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { | ||
| s"$deletedPartitions are gone. ${KafkaSourceProvider.CUSTOM_GROUP_ID_ERROR_MESSAGE}" | ||
| } else { | ||
| s"$deletedPartitions are gone. Some data may have been missed." | ||
|
|
@@ -172,10 +172,10 @@ private[kafka010] class KafkaMicroBatchStream( | |
| override def commit(end: Offset): Unit = {} | ||
|
|
||
| override def stop(): Unit = { | ||
| kafkaOffsetReader.close() | ||
| offsetReader.close() | ||
| } | ||
|
|
||
| override def toString(): String = s"KafkaV2[$kafkaOffsetReader]" | ||
| override def toString(): String = s"KafkaV2[$offsetReader]" | ||
|
|
||
| /** | ||
| * Read initial partition offsets from the checkpoint, or decide the offsets and write them to | ||
|
|
@@ -195,11 +195,11 @@ private[kafka010] class KafkaMicroBatchStream( | |
| metadataLog.get(0).getOrElse { | ||
| val offsets = startingOffsets match { | ||
| case EarliestOffsetRangeLimit => | ||
| KafkaSourceOffset(kafkaOffsetReader.fetchEarliestOffsets()) | ||
| KafkaSourceOffset(offsetReader.fetchEarliestOffsets()) | ||
| case LatestOffsetRangeLimit => | ||
| KafkaSourceOffset(kafkaOffsetReader.fetchLatestOffsets(None)) | ||
| KafkaSourceOffset(offsetReader.fetchLatestOffsets(None)) | ||
| case SpecificOffsetRangeLimit(p) => | ||
| kafkaOffsetReader.fetchSpecificOffsets(p, reportDataLoss) | ||
| offsetReader.fetchSpecificOffsets(p, reportDataLoss) | ||
| } | ||
| metadataLog.add(0, offsets) | ||
| logInfo(s"Initial offsets: $offsets") | ||
|
|
@@ -212,7 +212,7 @@ private[kafka010] class KafkaMicroBatchStream( | |
| limit: Long, | ||
| from: PartitionOffsetMap, | ||
| until: PartitionOffsetMap): PartitionOffsetMap = { | ||
| val fromNew = kafkaOffsetReader.fetchEarliestOffsets(until.keySet.diff(from.keySet).toSeq) | ||
| val fromNew = offsetReader.fetchEarliestOffsets(until.keySet.diff(from.keySet).toSeq) | ||
| val sizes = until.flatMap { | ||
| case (tp, end) => | ||
| // If begin isn't defined, something's wrong, but let alert logic in getBatch handle it | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -78,32 +78,32 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister | |
| schema: Option[StructType], | ||
| providerName: String, | ||
| parameters: Map[String, String]): Source = { | ||
| validateStreamOptions(parameters) | ||
| val caseInsensitiveParameters = CaseInsensitiveMap(parameters) | ||
| validateStreamOptions(caseInsensitiveParameters) | ||
| // 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 = streamingUniqueGroupId(parameters, metadataPath) | ||
| val uniqueGroupId = streamingUniqueGroupId(caseInsensitiveParameters, metadataPath) | ||
|
|
||
| val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } | ||
| val specifiedKafkaParams = convertToSpecifiedParams(parameters) | ||
|
|
||
| val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(caseInsensitiveParams, | ||
| STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) | ||
| val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( | ||
| caseInsensitiveParameters, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) | ||
|
|
||
| val kafkaOffsetReader = new KafkaOffsetReader( | ||
| strategy(caseInsensitiveParams), | ||
| strategy(caseInsensitiveParameters), | ||
| kafkaParamsForDriver(specifiedKafkaParams), | ||
| parameters, | ||
| caseInsensitiveParameters, | ||
| driverGroupIdPrefix = s"$uniqueGroupId-driver") | ||
|
|
||
| new KafkaSource( | ||
| sqlContext, | ||
| kafkaOffsetReader, | ||
| kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId), | ||
| parameters, | ||
| caseInsensitiveParameters, | ||
| metadataPath, | ||
| startingStreamOffsets, | ||
| failOnDataLoss(caseInsensitiveParams)) | ||
| failOnDataLoss(caseInsensitiveParameters)) | ||
| } | ||
|
|
||
| override def getTable(options: CaseInsensitiveStringMap): KafkaTable = { | ||
|
|
@@ -119,24 +119,24 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister | |
| override def createRelation( | ||
| sqlContext: SQLContext, | ||
| parameters: Map[String, String]): BaseRelation = { | ||
| validateBatchOptions(parameters) | ||
| val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } | ||
| val caseInsensitiveParameters = CaseInsensitiveMap(parameters) | ||
| validateBatchOptions(caseInsensitiveParameters) | ||
| val specifiedKafkaParams = convertToSpecifiedParams(parameters) | ||
|
|
||
| val startingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( | ||
| caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) | ||
| caseInsensitiveParameters, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) | ||
| assert(startingRelationOffsets != LatestOffsetRangeLimit) | ||
|
|
||
| val endingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(caseInsensitiveParams, | ||
| ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) | ||
| val endingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( | ||
| caseInsensitiveParameters, ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) | ||
| assert(endingRelationOffsets != EarliestOffsetRangeLimit) | ||
|
|
||
| new KafkaRelation( | ||
| sqlContext, | ||
| strategy(caseInsensitiveParams), | ||
| sourceOptions = parameters, | ||
| strategy(caseInsensitiveParameters), | ||
| sourceOptions = caseInsensitiveParameters, | ||
| specifiedKafkaParams = specifiedKafkaParams, | ||
| failOnDataLoss = failOnDataLoss(caseInsensitiveParams), | ||
| failOnDataLoss = failOnDataLoss(caseInsensitiveParameters), | ||
| startingOffsets = startingRelationOffsets, | ||
| endingOffsets = endingRelationOffsets) | ||
| } | ||
|
|
@@ -420,23 +420,22 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister | |
| } | ||
|
|
||
| override def toMicroBatchStream(checkpointLocation: String): MicroBatchStream = { | ||
| val parameters = options.asScala.toMap | ||
| validateStreamOptions(parameters) | ||
| val caseInsensitiveOptions = CaseInsensitiveMap(options.asScala.toMap) | ||
| validateStreamOptions(caseInsensitiveOptions) | ||
| // 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 = streamingUniqueGroupId(parameters, checkpointLocation) | ||
| val uniqueGroupId = streamingUniqueGroupId(caseInsensitiveOptions, checkpointLocation) | ||
|
|
||
| val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } | ||
| val specifiedKafkaParams = convertToSpecifiedParams(parameters) | ||
| val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveOptions) | ||
|
|
||
| val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( | ||
| caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) | ||
| caseInsensitiveOptions, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) | ||
|
|
||
| val kafkaOffsetReader = new KafkaOffsetReader( | ||
| strategy(parameters), | ||
| strategy(caseInsensitiveOptions), | ||
| kafkaParamsForDriver(specifiedKafkaParams), | ||
| parameters, | ||
| caseInsensitiveOptions, | ||
| driverGroupIdPrefix = s"$uniqueGroupId-driver") | ||
|
|
||
| new KafkaMicroBatchStream( | ||
|
|
@@ -445,32 +444,26 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister | |
| options, | ||
| checkpointLocation, | ||
| startingStreamOffsets, | ||
| failOnDataLoss(caseInsensitiveParams)) | ||
| failOnDataLoss(caseInsensitiveOptions)) | ||
| } | ||
|
|
||
| override def toContinuousStream(checkpointLocation: String): ContinuousStream = { | ||
| val parameters = options.asScala.toMap | ||
| validateStreamOptions(parameters) | ||
| val caseInsensitiveOptions = CaseInsensitiveMap(options.asScala.toMap) | ||
| validateStreamOptions(caseInsensitiveOptions) | ||
| // 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 = streamingUniqueGroupId(parameters, checkpointLocation) | ||
| val uniqueGroupId = streamingUniqueGroupId(caseInsensitiveOptions, checkpointLocation) | ||
|
|
||
| val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } | ||
| val specifiedKafkaParams = | ||
| parameters | ||
| .keySet | ||
| .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) | ||
| .map { k => k.drop(6).toString -> parameters(k) } | ||
| .toMap | ||
| val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveOptions) | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is not a required change but thought it would be good to simplify. |
||
|
|
||
| val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( | ||
| caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) | ||
| caseInsensitiveOptions, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) | ||
|
|
||
| val kafkaOffsetReader = new KafkaOffsetReader( | ||
| strategy(caseInsensitiveParams), | ||
| strategy(caseInsensitiveOptions), | ||
| kafkaParamsForDriver(specifiedKafkaParams), | ||
| parameters, | ||
| caseInsensitiveOptions, | ||
| driverGroupIdPrefix = s"$uniqueGroupId-driver") | ||
|
|
||
| new KafkaContinuousStream( | ||
|
|
@@ -479,7 +472,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister | |
| options, | ||
| checkpointLocation, | ||
| startingStreamOffsets, | ||
| failOnDataLoss(caseInsensitiveParams)) | ||
| failOnDataLoss(caseInsensitiveOptions)) | ||
| } | ||
| } | ||
| } | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.