17
17
18
18
package org .apache .spark .streaming .kafka010
19
19
20
- import java .{ util => ju }
20
+ import java .{ lang => jl , util => ju }
21
21
22
22
import scala .collection .JavaConverters ._
23
23
@@ -30,15 +30,16 @@ import org.apache.spark.annotation.Experimental
30
30
/**
31
31
* :: Experimental ::
32
32
* Choice of how to create and configure underlying Kafka Consumers on driver and executors.
33
+ * See [[ConsumerStrategies ]] to obtain instances.
33
34
* Kafka 0.10 consumers can require additional, sometimes complex, setup after object
34
35
* instantiation. This interface encapsulates that process, and allows it to be checkpointed.
35
36
* @tparam K type of Kafka message key
36
37
* @tparam V type of Kafka message value
37
38
*/
38
39
@ Experimental
39
- trait ConsumerStrategy [K , V ] {
40
+ abstract class ConsumerStrategy [K , V ] {
40
41
/**
41
- * Kafka <a href="http://kafka.apache.org/documentation.htmll #newconsumerconfigs">
42
+ * Kafka <a href="http://kafka.apache.org/documentation.html #newconsumerconfigs">
42
43
* configuration parameters</a> to be used on executors. Requires "bootstrap.servers" to be set
43
44
* with Kafka broker(s) specified in host1:port1,host2:port2 form.
44
45
*/
@@ -51,15 +52,14 @@ trait ConsumerStrategy[K, V] {
51
52
* has successfully read. Will be empty on initial start, possibly non-empty on restart from
52
53
* checkpoint.
53
54
*/
54
- def onStart (currentOffsets : Map [TopicPartition , Long ]): Consumer [K , V ]
55
+ def onStart (currentOffsets : ju. Map [TopicPartition , jl. Long ]): Consumer [K , V ]
55
56
}
56
57
57
58
/**
58
- * :: Experimental ::
59
59
* Subscribe to a collection of topics.
60
60
* @param topics collection of topics to subscribe
61
61
* @param kafkaParams Kafka
62
- * <a href="http://kafka.apache.org/documentation.htmll #newconsumerconfigs">
62
+ * <a href="http://kafka.apache.org/documentation.html #newconsumerconfigs">
63
63
* configuration parameters</a> to be used on driver. The same params will be used on executors,
64
64
* with minor automatic modifications applied.
65
65
* Requires "bootstrap.servers" to be set
@@ -68,16 +68,15 @@ trait ConsumerStrategy[K, V] {
68
68
* TopicPartition, the committed offset (if applicable) or kafka param
69
69
* auto.offset.reset will be used.
70
70
*/
71
- @ Experimental
72
- case class Subscribe [K , V ] private (
73
- topics : ju.Collection [java.lang.String ],
71
+ private case class Subscribe [K , V ](
72
+ topics : ju.Collection [jl.String ],
74
73
kafkaParams : ju.Map [String , Object ],
75
- offsets : ju.Map [TopicPartition , Long ]
74
+ offsets : ju.Map [TopicPartition , jl. Long ]
76
75
) extends ConsumerStrategy [K , V ] {
77
76
78
77
def executorKafkaParams : ju.Map [String , Object ] = kafkaParams
79
78
80
- def onStart (currentOffsets : Map [TopicPartition , Long ]): Consumer [K , V ] = {
79
+ def onStart (currentOffsets : ju. Map [TopicPartition , jl. Long ]): Consumer [K , V ] = {
81
80
val consumer = new KafkaConsumer [K , V ](kafkaParams)
82
81
consumer.subscribe(topics)
83
82
if (currentOffsets.isEmpty) {
@@ -90,18 +89,52 @@ case class Subscribe[K, V] private(
90
89
}
91
90
}
92
91
92
+ /**
93
+ * Assign a fixed collection of TopicPartitions
94
+ * @param topicPartitions collection of TopicPartitions to assign
95
+ * @param kafkaParams Kafka
96
+ * <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
97
+ * configuration parameters</a> to be used on driver. The same params will be used on executors,
98
+ * with minor automatic modifications applied.
99
+ * Requires "bootstrap.servers" to be set
100
+ * with Kafka broker(s) specified in host1:port1,host2:port2 form.
101
+ * @param offsets: offsets to begin at on initial startup. If no offset is given for a
102
+ * TopicPartition, the committed offset (if applicable) or kafka param
103
+ * auto.offset.reset will be used.
104
+ */
105
+ private case class Assign [K , V ](
106
+ topicPartitions : ju.Collection [TopicPartition ],
107
+ kafkaParams : ju.Map [String , Object ],
108
+ offsets : ju.Map [TopicPartition , jl.Long ]
109
+ ) extends ConsumerStrategy [K , V ] {
110
+
111
+ def executorKafkaParams : ju.Map [String , Object ] = kafkaParams
112
+
113
+ def onStart (currentOffsets : ju.Map [TopicPartition , jl.Long ]): Consumer [K , V ] = {
114
+ val consumer = new KafkaConsumer [K , V ](kafkaParams)
115
+ consumer.assign(topicPartitions)
116
+ if (currentOffsets.isEmpty) {
117
+ offsets.asScala.foreach { case (topicPartition, offset) =>
118
+ consumer.seek(topicPartition, offset)
119
+ }
120
+ }
121
+
122
+ consumer
123
+ }
124
+ }
125
+
93
126
/**
94
127
* :: Experimental ::
95
- * Companion object for creating [[ Subscribe ]] strategy
128
+ * object for obtaining instances of [[ ConsumerStrategy ]]
96
129
*/
97
130
@ Experimental
98
- object Subscribe {
131
+ object ConsumerStrategies {
99
132
/**
100
133
* :: Experimental ::
101
134
* Subscribe to a collection of topics.
102
135
* @param topics collection of topics to subscribe
103
136
* @param kafkaParams Kafka
104
- * <a href="http://kafka.apache.org/documentation.htmll #newconsumerconfigs">
137
+ * <a href="http://kafka.apache.org/documentation.html #newconsumerconfigs">
105
138
* configuration parameters</a> to be used on driver. The same params will be used on executors,
106
139
* with minor automatic modifications applied.
107
140
* Requires "bootstrap.servers" to be set
@@ -111,43 +144,43 @@ object Subscribe {
111
144
* auto.offset.reset will be used.
112
145
*/
113
146
@ Experimental
114
- def apply [K , V ](
115
- topics : Iterable [java.lang .String ],
147
+ def Subscribe [K , V ](
148
+ topics : Iterable [jl .String ],
116
149
kafkaParams : collection.Map [String , Object ],
117
- offsets : collection.Map [TopicPartition , Long ]): Subscribe [K , V ] = {
118
- Subscribe [K , V ](
150
+ offsets : collection.Map [TopicPartition , Long ]): ConsumerStrategy [K , V ] = {
151
+ new Subscribe [K , V ](
119
152
new ju.ArrayList (topics.asJavaCollection),
120
153
new ju.HashMap [String , Object ](kafkaParams.asJava),
121
- new ju.HashMap [TopicPartition , Long ](offsets.asJava))
154
+ new ju.HashMap [TopicPartition , jl. Long ](offsets.mapValues(l => new jl. Long (l)) .asJava))
122
155
}
123
156
124
157
/**
125
158
* :: Experimental ::
126
159
* Subscribe to a collection of topics.
127
160
* @param topics collection of topics to subscribe
128
161
* @param kafkaParams Kafka
129
- * <a href="http://kafka.apache.org/documentation.htmll #newconsumerconfigs">
162
+ * <a href="http://kafka.apache.org/documentation.html #newconsumerconfigs">
130
163
* configuration parameters</a> to be used on driver. The same params will be used on executors,
131
164
* with minor automatic modifications applied.
132
165
* Requires "bootstrap.servers" to be set
133
166
* with Kafka broker(s) specified in host1:port1,host2:port2 form.
134
167
*/
135
168
@ Experimental
136
- def apply [K , V ](
137
- topics : Iterable [java.lang .String ],
138
- kafkaParams : collection.Map [String , Object ]): Subscribe [K , V ] = {
139
- Subscribe [K , V ](
169
+ def Subscribe [K , V ](
170
+ topics : Iterable [jl .String ],
171
+ kafkaParams : collection.Map [String , Object ]): ConsumerStrategy [K , V ] = {
172
+ new Subscribe [K , V ](
140
173
new ju.ArrayList (topics.asJavaCollection),
141
174
new ju.HashMap [String , Object ](kafkaParams.asJava),
142
- ju.Collections .emptyMap[TopicPartition , Long ]())
175
+ ju.Collections .emptyMap[TopicPartition , jl. Long ]())
143
176
}
144
177
145
178
/**
146
179
* :: Experimental ::
147
180
* Subscribe to a collection of topics.
148
181
* @param topics collection of topics to subscribe
149
182
* @param kafkaParams Kafka
150
- * <a href="http://kafka.apache.org/documentation.htmll #newconsumerconfigs">
183
+ * <a href="http://kafka.apache.org/documentation.html #newconsumerconfigs">
151
184
* configuration parameters</a> to be used on driver. The same params will be used on executors,
152
185
* with minor automatic modifications applied.
153
186
* Requires "bootstrap.servers" to be set
@@ -157,81 +190,37 @@ object Subscribe {
157
190
* auto.offset.reset will be used.
158
191
*/
159
192
@ Experimental
160
- def create [K , V ](
161
- topics : ju.Collection [java.lang .String ],
193
+ def Subscribe [K , V ](
194
+ topics : ju.Collection [jl .String ],
162
195
kafkaParams : ju.Map [String , Object ],
163
- offsets : ju.Map [TopicPartition , Long ]): Subscribe [K , V ] = {
164
- Subscribe [K , V ](topics, kafkaParams, offsets)
196
+ offsets : ju.Map [TopicPartition , jl. Long ]): ConsumerStrategy [K , V ] = {
197
+ new Subscribe [K , V ](topics, kafkaParams, offsets)
165
198
}
166
199
167
200
/**
168
201
* :: Experimental ::
169
202
* Subscribe to a collection of topics.
170
203
* @param topics collection of topics to subscribe
171
204
* @param kafkaParams Kafka
172
- * <a href="http://kafka.apache.org/documentation.htmll #newconsumerconfigs">
205
+ * <a href="http://kafka.apache.org/documentation.html #newconsumerconfigs">
173
206
* configuration parameters</a> to be used on driver. The same params will be used on executors,
174
207
* with minor automatic modifications applied.
175
208
* Requires "bootstrap.servers" to be set
176
209
* with Kafka broker(s) specified in host1:port1,host2:port2 form.
177
210
*/
178
211
@ Experimental
179
- def create [K , V ](
180
- topics : ju.Collection [java.lang.String ],
181
- kafkaParams : ju.Map [String , Object ]): Subscribe [K , V ] = {
182
- Subscribe [K , V ](topics, kafkaParams, ju.Collections .emptyMap[TopicPartition , Long ]())
183
- }
184
-
185
- }
186
-
187
- /**
188
- * :: Experimental ::
189
- * Assign a fixed collection of TopicPartitions
190
- * @param topicPartitions collection of TopicPartitions to assign
191
- * @param kafkaParams Kafka
192
- * <a href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs">
193
- * configuration parameters</a> to be used on driver. The same params will be used on executors,
194
- * with minor automatic modifications applied.
195
- * Requires "bootstrap.servers" to be set
196
- * with Kafka broker(s) specified in host1:port1,host2:port2 form.
197
- * @param offsets: offsets to begin at on initial startup. If no offset is given for a
198
- * TopicPartition, the committed offset (if applicable) or kafka param
199
- * auto.offset.reset will be used.
200
- */
201
- @ Experimental
202
- case class Assign [K , V ] private (
203
- topicPartitions : ju.Collection [TopicPartition ],
204
- kafkaParams : ju.Map [String , Object ],
205
- offsets : ju.Map [TopicPartition , Long ]
206
- ) extends ConsumerStrategy [K , V ] {
207
-
208
- def executorKafkaParams : ju.Map [String , Object ] = kafkaParams
209
-
210
- def onStart (currentOffsets : Map [TopicPartition , Long ]): Consumer [K , V ] = {
211
- val consumer = new KafkaConsumer [K , V ](kafkaParams)
212
- consumer.assign(topicPartitions)
213
- if (currentOffsets.isEmpty) {
214
- offsets.asScala.foreach { case (topicPartition, offset) =>
215
- consumer.seek(topicPartition, offset)
216
- }
217
- }
218
-
219
- consumer
212
+ def Subscribe [K , V ](
213
+ topics : ju.Collection [jl.String ],
214
+ kafkaParams : ju.Map [String , Object ]): ConsumerStrategy [K , V ] = {
215
+ new Subscribe [K , V ](topics, kafkaParams, ju.Collections .emptyMap[TopicPartition , jl.Long ]())
220
216
}
221
- }
222
217
223
- /**
224
- * :: Experimental ::
225
- * Companion object for creating [[Assign ]] strategy
226
- */
227
- @ Experimental
228
- object Assign {
229
218
/**
230
219
* :: Experimental ::
231
220
* Assign a fixed collection of TopicPartitions
232
221
* @param topicPartitions collection of TopicPartitions to assign
233
222
* @param kafkaParams Kafka
234
- * <a href="http://kafka.apache.org/documentation.htmll #newconsumerconfigs">
223
+ * <a href="http://kafka.apache.org/documentation.html #newconsumerconfigs">
235
224
* configuration parameters</a> to be used on driver. The same params will be used on executors,
236
225
* with minor automatic modifications applied.
237
226
* Requires "bootstrap.servers" to be set
@@ -241,43 +230,43 @@ object Assign {
241
230
* auto.offset.reset will be used.
242
231
*/
243
232
@ Experimental
244
- def apply [K , V ](
233
+ def Assign [K , V ](
245
234
topicPartitions : Iterable [TopicPartition ],
246
235
kafkaParams : collection.Map [String , Object ],
247
- offsets : collection.Map [TopicPartition , Long ]): Assign [K , V ] = {
248
- Assign [K , V ](
236
+ offsets : collection.Map [TopicPartition , Long ]): ConsumerStrategy [K , V ] = {
237
+ new Assign [K , V ](
249
238
new ju.ArrayList (topicPartitions.asJavaCollection),
250
239
new ju.HashMap [String , Object ](kafkaParams.asJava),
251
- new ju.HashMap [TopicPartition , Long ](offsets.asJava))
240
+ new ju.HashMap [TopicPartition , jl. Long ](offsets.mapValues(l => new jl. Long (l)) .asJava))
252
241
}
253
242
254
243
/**
255
244
* :: Experimental ::
256
245
* Assign a fixed collection of TopicPartitions
257
246
* @param topicPartitions collection of TopicPartitions to assign
258
247
* @param kafkaParams Kafka
259
- * <a href="http://kafka.apache.org/documentation.htmll #newconsumerconfigs">
248
+ * <a href="http://kafka.apache.org/documentation.html #newconsumerconfigs">
260
249
* configuration parameters</a> to be used on driver. The same params will be used on executors,
261
250
* with minor automatic modifications applied.
262
251
* Requires "bootstrap.servers" to be set
263
252
* with Kafka broker(s) specified in host1:port1,host2:port2 form.
264
253
*/
265
254
@ Experimental
266
- def apply [K , V ](
255
+ def Assign [K , V ](
267
256
topicPartitions : Iterable [TopicPartition ],
268
- kafkaParams : collection.Map [String , Object ]): Assign [K , V ] = {
269
- Assign [K , V ](
257
+ kafkaParams : collection.Map [String , Object ]): ConsumerStrategy [K , V ] = {
258
+ new Assign [K , V ](
270
259
new ju.ArrayList (topicPartitions.asJavaCollection),
271
260
new ju.HashMap [String , Object ](kafkaParams.asJava),
272
- ju.Collections .emptyMap[TopicPartition , Long ]())
261
+ ju.Collections .emptyMap[TopicPartition , jl. Long ]())
273
262
}
274
263
275
264
/**
276
265
* :: Experimental ::
277
266
* Assign a fixed collection of TopicPartitions
278
267
* @param topicPartitions collection of TopicPartitions to assign
279
268
* @param kafkaParams Kafka
280
- * <a href="http://kafka.apache.org/documentation.htmll #newconsumerconfigs">
269
+ * <a href="http://kafka.apache.org/documentation.html #newconsumerconfigs">
281
270
* configuration parameters</a> to be used on driver. The same params will be used on executors,
282
271
* with minor automatic modifications applied.
283
272
* Requires "bootstrap.servers" to be set
@@ -287,28 +276,32 @@ object Assign {
287
276
* auto.offset.reset will be used.
288
277
*/
289
278
@ Experimental
290
- def create [K , V ](
279
+ def Assign [K , V ](
291
280
topicPartitions : ju.Collection [TopicPartition ],
292
281
kafkaParams : ju.Map [String , Object ],
293
- offsets : ju.Map [TopicPartition , Long ]): Assign [K , V ] = {
294
- Assign [K , V ](topicPartitions, kafkaParams, offsets)
282
+ offsets : ju.Map [TopicPartition , jl. Long ]): ConsumerStrategy [K , V ] = {
283
+ new Assign [K , V ](topicPartitions, kafkaParams, offsets)
295
284
}
296
285
297
286
/**
298
287
* :: Experimental ::
299
288
* Assign a fixed collection of TopicPartitions
300
289
* @param topicPartitions collection of TopicPartitions to assign
301
290
* @param kafkaParams Kafka
302
- * <a href="http://kafka.apache.org/documentation.htmll #newconsumerconfigs">
291
+ * <a href="http://kafka.apache.org/documentation.html #newconsumerconfigs">
303
292
* configuration parameters</a> to be used on driver. The same params will be used on executors,
304
293
* with minor automatic modifications applied.
305
294
* Requires "bootstrap.servers" to be set
306
295
* with Kafka broker(s) specified in host1:port1,host2:port2 form.
307
296
*/
308
297
@ Experimental
309
- def create [K , V ](
298
+ def Assign [K , V ](
310
299
topicPartitions : ju.Collection [TopicPartition ],
311
- kafkaParams : ju.Map [String , Object ]): Assign [K , V ] = {
312
- Assign [K , V ](topicPartitions, kafkaParams, ju.Collections .emptyMap[TopicPartition , Long ]())
300
+ kafkaParams : ju.Map [String , Object ]): ConsumerStrategy [K , V ] = {
301
+ new Assign [K , V ](
302
+ topicPartitions,
303
+ kafkaParams,
304
+ ju.Collections .emptyMap[TopicPartition , jl.Long ]())
313
305
}
306
+
314
307
}
0 commit comments