@@ -111,10 +111,6 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
111111 sqlContext : SQLContext ,
112112 parameters : Map [String , String ]): BaseRelation = {
113113 validateBatchOptions(parameters)
114- // Each running query should use its own group id. Otherwise, the query may be only assigned
115- // partial data since Kafka will assign partitions to multiple consumers having the same group
116- // id. Hence, we should generate a unique id for each query.
117- val uniqueGroupId = s " spark-kafka-relation- ${UUID .randomUUID}"
118114 val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale .ROOT ), v) }
119115 val specifiedKafkaParams =
120116 parameters
@@ -131,20 +127,14 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
131127 ENDING_OFFSETS_OPTION_KEY , LatestOffsetRangeLimit )
132128 assert(endingRelationOffsets != EarliestOffsetRangeLimit )
133129
134- val kafkaOffsetReader = new KafkaOffsetReader (
135- strategy(caseInsensitiveParams),
136- kafkaParamsForDriver(specifiedKafkaParams),
137- parameters,
138- driverGroupIdPrefix = s " $uniqueGroupId-driver " )
139-
140130 new KafkaRelation (
141131 sqlContext,
142- kafkaOffsetReader ,
143- kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId) ,
144- parameters ,
145- failOnDataLoss(caseInsensitiveParams),
146- startingRelationOffsets,
147- endingRelationOffsets)
132+ strategy(caseInsensitiveParams) ,
133+ sourceOptions = parameters ,
134+ specifiedKafkaParams = specifiedKafkaParams ,
135+ failOnDataLoss = failOnDataLoss (caseInsensitiveParams),
136+ startingOffsets = startingRelationOffsets,
137+ endingOffsets = endingRelationOffsets)
148138 }
149139
150140 override def createSink (
@@ -213,46 +203,6 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
213203 ProducerConfig .VALUE_SERIALIZER_CLASS_CONFIG -> classOf [ByteArraySerializer ].getName)
214204 }
215205
216- private def kafkaParamsForDriver (specifiedKafkaParams : Map [String , String ]) =
217- ConfigUpdater (" source" , specifiedKafkaParams)
218- .set(ConsumerConfig .KEY_DESERIALIZER_CLASS_CONFIG , deserClassName)
219- .set(ConsumerConfig .VALUE_DESERIALIZER_CLASS_CONFIG , deserClassName)
220-
221- // Set to "earliest" to avoid exceptions. However, KafkaSource will fetch the initial
222- // offsets by itself instead of counting on KafkaConsumer.
223- .set(ConsumerConfig .AUTO_OFFSET_RESET_CONFIG , " earliest" )
224-
225- // So that consumers in the driver does not commit offsets unnecessarily
226- .set(ConsumerConfig .ENABLE_AUTO_COMMIT_CONFIG , " false" )
227-
228- // So that the driver does not pull too much data
229- .set(ConsumerConfig .MAX_POLL_RECORDS_CONFIG , new java.lang.Integer (1 ))
230-
231- // If buffer config is not set, set it to reasonable value to work around
232- // buffer issues (see KAFKA-3135)
233- .setIfUnset(ConsumerConfig .RECEIVE_BUFFER_CONFIG , 65536 : java.lang.Integer )
234- .build()
235-
236- private def kafkaParamsForExecutors (
237- specifiedKafkaParams : Map [String , String ], uniqueGroupId : String ) =
238- ConfigUpdater (" executor" , specifiedKafkaParams)
239- .set(ConsumerConfig .KEY_DESERIALIZER_CLASS_CONFIG , deserClassName)
240- .set(ConsumerConfig .VALUE_DESERIALIZER_CLASS_CONFIG , deserClassName)
241-
242- // Make sure executors do only what the driver tells them.
243- .set(ConsumerConfig .AUTO_OFFSET_RESET_CONFIG , " none" )
244-
245- // So that consumers in executors do not mess with any existing group id
246- .set(ConsumerConfig .GROUP_ID_CONFIG , s " $uniqueGroupId-executor " )
247-
248- // So that consumers in executors does not commit offsets unnecessarily
249- .set(ConsumerConfig .ENABLE_AUTO_COMMIT_CONFIG , " false" )
250-
251- // If buffer config is not set, set it to reasonable value to work around
252- // buffer issues (see KAFKA-3135)
253- .setIfUnset(ConsumerConfig .RECEIVE_BUFFER_CONFIG , 65536 : java.lang.Integer )
254- .build()
255-
256206 private def strategy (caseInsensitiveParams : Map [String , String ]) =
257207 caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS .contains(x._1)).get match {
258208 case (" assign" , value) =>
@@ -414,30 +364,9 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
414364 logWarning(" maxOffsetsPerTrigger option ignored in batch queries" )
415365 }
416366 }
417-
418- /** Class to conveniently update Kafka config params, while logging the changes */
419- private case class ConfigUpdater (module : String , kafkaParams : Map [String , String ]) {
420- private val map = new ju.HashMap [String , Object ](kafkaParams.asJava)
421-
422- def set (key : String , value : Object ): this .type = {
423- map.put(key, value)
424- logInfo(s " $module: Set $key to $value, earlier value: ${kafkaParams.getOrElse(key, " " )}" )
425- this
426- }
427-
428- def setIfUnset (key : String , value : Object ): ConfigUpdater = {
429- if (! map.containsKey(key)) {
430- map.put(key, value)
431- logInfo(s " $module: Set $key to $value" )
432- }
433- this
434- }
435-
436- def build (): ju.Map [String , Object ] = map
437- }
438367}
439368
440- private [kafka010] object KafkaSourceProvider {
369+ private [kafka010] object KafkaSourceProvider extends Logging {
441370 private val STRATEGY_OPTION_KEYS = Set (" subscribe" , " subscribepattern" , " assign" )
442371 private [kafka010] val STARTING_OFFSETS_OPTION_KEY = " startingoffsets"
443372 private [kafka010] val ENDING_OFFSETS_OPTION_KEY = " endingoffsets"
@@ -459,4 +388,66 @@ private[kafka010] object KafkaSourceProvider {
459388 case None => defaultOffsets
460389 }
461390 }
391+
392+ def kafkaParamsForDriver (specifiedKafkaParams : Map [String , String ]): ju.Map [String , Object ] =
393+ ConfigUpdater (" source" , specifiedKafkaParams)
394+ .set(ConsumerConfig .KEY_DESERIALIZER_CLASS_CONFIG , deserClassName)
395+ .set(ConsumerConfig .VALUE_DESERIALIZER_CLASS_CONFIG , deserClassName)
396+
397+ // Set to "earliest" to avoid exceptions. However, KafkaSource will fetch the initial
398+ // offsets by itself instead of counting on KafkaConsumer.
399+ .set(ConsumerConfig .AUTO_OFFSET_RESET_CONFIG , " earliest" )
400+
401+ // So that consumers in the driver does not commit offsets unnecessarily
402+ .set(ConsumerConfig .ENABLE_AUTO_COMMIT_CONFIG , " false" )
403+
404+ // So that the driver does not pull too much data
405+ .set(ConsumerConfig .MAX_POLL_RECORDS_CONFIG , new java.lang.Integer (1 ))
406+
407+ // If buffer config is not set, set it to reasonable value to work around
408+ // buffer issues (see KAFKA-3135)
409+ .setIfUnset(ConsumerConfig .RECEIVE_BUFFER_CONFIG , 65536 : java.lang.Integer )
410+ .build()
411+
412+ def kafkaParamsForExecutors (
413+ specifiedKafkaParams : Map [String , String ],
414+ uniqueGroupId : String ): ju.Map [String , Object ] =
415+ ConfigUpdater (" executor" , specifiedKafkaParams)
416+ .set(ConsumerConfig .KEY_DESERIALIZER_CLASS_CONFIG , deserClassName)
417+ .set(ConsumerConfig .VALUE_DESERIALIZER_CLASS_CONFIG , deserClassName)
418+
419+ // Make sure executors do only what the driver tells them.
420+ .set(ConsumerConfig .AUTO_OFFSET_RESET_CONFIG , " none" )
421+
422+ // So that consumers in executors do not mess with any existing group id
423+ .set(ConsumerConfig .GROUP_ID_CONFIG , s " $uniqueGroupId-executor " )
424+
425+ // So that consumers in executors does not commit offsets unnecessarily
426+ .set(ConsumerConfig .ENABLE_AUTO_COMMIT_CONFIG , " false" )
427+
428+ // If buffer config is not set, set it to reasonable value to work around
429+ // buffer issues (see KAFKA-3135)
430+ .setIfUnset(ConsumerConfig .RECEIVE_BUFFER_CONFIG , 65536 : java.lang.Integer )
431+ .build()
432+
433+ /** Class to conveniently update Kafka config params, while logging the changes */
434+ private case class ConfigUpdater (module : String , kafkaParams : Map [String , String ]) {
435+ private val map = new ju.HashMap [String , Object ](kafkaParams.asJava)
436+
437+ def set (key : String , value : Object ): this .type = {
438+ map.put(key, value)
439+ logDebug(s " $module: Set $key to $value, earlier value: ${kafkaParams.getOrElse(key, " " )}" )
440+ this
441+ }
442+
443+ def setIfUnset (key : String , value : Object ): ConfigUpdater = {
444+ if (! map.containsKey(key)) {
445+ map.put(key, value)
446+ logDebug(s " $module: Set $key to $value" )
447+ }
448+ this
449+ }
450+
451+ def build (): ju.Map [String , Object ] = map
452+ }
462453}
0 commit comments