@@ -78,32 +78,32 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
7878 schema : Option [StructType ],
7979 providerName : String ,
8080 parameters : Map [String , String ]): Source = {
81- validateStreamOptions(parameters)
81+ val caseInsensitiveParameters = CaseInsensitiveMap (parameters)
82+ validateStreamOptions(caseInsensitiveParameters)
8283 // Each running query should use its own group id. Otherwise, the query may be only assigned
8384 // partial data since Kafka will assign partitions to multiple consumers having the same group
8485 // id. Hence, we should generate a unique id for each query.
85- val uniqueGroupId = streamingUniqueGroupId(parameters , metadataPath)
86+ val uniqueGroupId = streamingUniqueGroupId(caseInsensitiveParameters , metadataPath)
8687
87- val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale .ROOT ), v) }
8888 val specifiedKafkaParams = convertToSpecifiedParams(parameters)
8989
90- val startingStreamOffsets = KafkaSourceProvider .getKafkaOffsetRangeLimit(caseInsensitiveParams,
91- STARTING_OFFSETS_OPTION_KEY , LatestOffsetRangeLimit )
90+ val startingStreamOffsets = KafkaSourceProvider .getKafkaOffsetRangeLimit(
91+ caseInsensitiveParameters, STARTING_OFFSETS_OPTION_KEY , LatestOffsetRangeLimit )
9292
9393 val kafkaOffsetReader = new KafkaOffsetReader (
94- strategy(caseInsensitiveParams ),
94+ strategy(caseInsensitiveParameters ),
9595 kafkaParamsForDriver(specifiedKafkaParams),
96- parameters ,
96+ caseInsensitiveParameters ,
9797 driverGroupIdPrefix = s " $uniqueGroupId-driver " )
9898
9999 new KafkaSource (
100100 sqlContext,
101101 kafkaOffsetReader,
102102 kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId),
103- parameters ,
103+ caseInsensitiveParameters ,
104104 metadataPath,
105105 startingStreamOffsets,
106- failOnDataLoss(caseInsensitiveParams ))
106+ failOnDataLoss(caseInsensitiveParameters ))
107107 }
108108
109109 override def getTable (options : CaseInsensitiveStringMap ): KafkaTable = {
@@ -119,24 +119,24 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
119119 override def createRelation (
120120 sqlContext : SQLContext ,
121121 parameters : Map [String , String ]): BaseRelation = {
122- validateBatchOptions (parameters)
123- val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase( Locale . ROOT ), v) }
122+ val caseInsensitiveParameters = CaseInsensitiveMap (parameters)
123+ validateBatchOptions(caseInsensitiveParameters)
124124 val specifiedKafkaParams = convertToSpecifiedParams(parameters)
125125
126126 val startingRelationOffsets = KafkaSourceProvider .getKafkaOffsetRangeLimit(
127- caseInsensitiveParams , STARTING_OFFSETS_OPTION_KEY , EarliestOffsetRangeLimit )
127+ caseInsensitiveParameters , STARTING_OFFSETS_OPTION_KEY , EarliestOffsetRangeLimit )
128128 assert(startingRelationOffsets != LatestOffsetRangeLimit )
129129
130- val endingRelationOffsets = KafkaSourceProvider .getKafkaOffsetRangeLimit(caseInsensitiveParams,
131- ENDING_OFFSETS_OPTION_KEY , LatestOffsetRangeLimit )
130+ val endingRelationOffsets = KafkaSourceProvider .getKafkaOffsetRangeLimit(
131+ caseInsensitiveParameters, ENDING_OFFSETS_OPTION_KEY , LatestOffsetRangeLimit )
132132 assert(endingRelationOffsets != EarliestOffsetRangeLimit )
133133
134134 new KafkaRelation (
135135 sqlContext,
136- strategy(caseInsensitiveParams ),
137- sourceOptions = parameters ,
136+ strategy(caseInsensitiveParameters ),
137+ sourceOptions = caseInsensitiveParameters ,
138138 specifiedKafkaParams = specifiedKafkaParams,
139- failOnDataLoss = failOnDataLoss(caseInsensitiveParams ),
139+ failOnDataLoss = failOnDataLoss(caseInsensitiveParameters ),
140140 startingOffsets = startingRelationOffsets,
141141 endingOffsets = endingRelationOffsets)
142142 }
@@ -420,23 +420,22 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
420420 }
421421
422422 override def toMicroBatchStream (checkpointLocation : String ): MicroBatchStream = {
423- val parameters = options.asScala.toMap
424- validateStreamOptions(parameters )
423+ val caseInsensitiveOptions = CaseInsensitiveMap ( options.asScala.toMap)
424+ validateStreamOptions(caseInsensitiveOptions )
425425 // Each running query should use its own group id. Otherwise, the query may be only assigned
426426 // partial data since Kafka will assign partitions to multiple consumers having the same group
427427 // id. Hence, we should generate a unique id for each query.
428- val uniqueGroupId = streamingUniqueGroupId(parameters , checkpointLocation)
428+ val uniqueGroupId = streamingUniqueGroupId(caseInsensitiveOptions , checkpointLocation)
429429
430- val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale .ROOT ), v) }
431- val specifiedKafkaParams = convertToSpecifiedParams(parameters)
430+ val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveOptions)
432431
433432 val startingStreamOffsets = KafkaSourceProvider .getKafkaOffsetRangeLimit(
434- caseInsensitiveParams , STARTING_OFFSETS_OPTION_KEY , LatestOffsetRangeLimit )
433+ caseInsensitiveOptions , STARTING_OFFSETS_OPTION_KEY , LatestOffsetRangeLimit )
435434
436435 val kafkaOffsetReader = new KafkaOffsetReader (
437- strategy(parameters ),
436+ strategy(caseInsensitiveOptions ),
438437 kafkaParamsForDriver(specifiedKafkaParams),
439- parameters ,
438+ caseInsensitiveOptions ,
440439 driverGroupIdPrefix = s " $uniqueGroupId-driver " )
441440
442441 new KafkaMicroBatchStream (
@@ -445,32 +444,26 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
445444 options,
446445 checkpointLocation,
447446 startingStreamOffsets,
448- failOnDataLoss(caseInsensitiveParams ))
447+ failOnDataLoss(caseInsensitiveOptions ))
449448 }
450449
451450 override def toContinuousStream (checkpointLocation : String ): ContinuousStream = {
452- val parameters = options.asScala.toMap
453- validateStreamOptions(parameters )
451+ val caseInsensitiveOptions = CaseInsensitiveMap ( options.asScala.toMap)
452+ validateStreamOptions(caseInsensitiveOptions )
454453 // Each running query should use its own group id. Otherwise, the query may be only assigned
455454 // partial data since Kafka will assign partitions to multiple consumers having the same group
456455 // id. Hence, we should generate a unique id for each query.
457- val uniqueGroupId = streamingUniqueGroupId(parameters , checkpointLocation)
456+ val uniqueGroupId = streamingUniqueGroupId(caseInsensitiveOptions , checkpointLocation)
458457
459- val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale .ROOT ), v) }
460- val specifiedKafkaParams =
461- parameters
462- .keySet
463- .filter(_.toLowerCase(Locale .ROOT ).startsWith(" kafka." ))
464- .map { k => k.drop(6 ).toString -> parameters(k) }
465- .toMap
458+ val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveOptions)
466459
467460 val startingStreamOffsets = KafkaSourceProvider .getKafkaOffsetRangeLimit(
468- caseInsensitiveParams , STARTING_OFFSETS_OPTION_KEY , LatestOffsetRangeLimit )
461+ caseInsensitiveOptions , STARTING_OFFSETS_OPTION_KEY , LatestOffsetRangeLimit )
469462
470463 val kafkaOffsetReader = new KafkaOffsetReader (
471- strategy(caseInsensitiveParams ),
464+ strategy(caseInsensitiveOptions ),
472465 kafkaParamsForDriver(specifiedKafkaParams),
473- parameters ,
466+ caseInsensitiveOptions ,
474467 driverGroupIdPrefix = s " $uniqueGroupId-driver " )
475468
476469 new KafkaContinuousStream (
@@ -479,7 +472,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
479472 options,
480473 checkpointLocation,
481474 startingStreamOffsets,
482- failOnDataLoss(caseInsensitiveParams ))
475+ failOnDataLoss(caseInsensitiveOptions ))
483476 }
484477 }
485478}
0 commit comments