-
Notifications
You must be signed in to change notification settings - Fork 29.1k
[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
Closed
Closed
Changes from all commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
23c2d71
[SPARK-28163][SS] Use CaseInsensitiveMap for KafkaOffsetReader
gaborgsomogyi d319e5b
Merge branch 'master' into SPARK-28163
gaborgsomogyi a4790ad
Added tests + converting incoming Map to CaseInsensitiveMap
gaborgsomogyi d20b85a
Change expected
gaborgsomogyi 3723930
Test cleanup :)
gaborgsomogyi dcc9d56
Rollback KafkaOffsetReader reanmes
gaborgsomogyi 5cd7b8e
Rollback KafkaMicroBatchStream reanmes
gaborgsomogyi 06fe0fc
Making offsetReader private[kafka010]
gaborgsomogyi 9034894
Remove private method testing approach
gaborgsomogyi e0e8a88
Compile fix
gaborgsomogyi File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| 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)) | ||
| } | ||
| } | ||
| } | ||
|
|
||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.