1717
1818package org .apache .spark .streaming .kafka09
1919
20- import java .{ util => ju }
20+ import java .{util => ju }
21+ import java .io .OutputStream
22+ import java .lang .{Integer => JInt , Long => JLong }
2123
24+ import com .google .common .base .Charsets .UTF_8
25+ import net .razorvine .pickle .{IObjectPickler , Opcodes , Pickler }
2226import org .apache .kafka .clients .consumer ._
2327import org .apache .kafka .common .TopicPartition
2428
2529import org .apache .spark .SparkContext
2630import org .apache .spark .annotation .Experimental
27- import org .apache .spark .api .java .{ JavaRDD , JavaSparkContext }
28- import org .apache .spark .api .java .function .{ Function0 => JFunction0 }
31+ import org .apache .spark .api .java .{JavaRDD , JavaSparkContext }
32+ import org .apache .spark .api .java .function .{Function0 => JFunction0 }
33+ import org .apache .spark .api .python .SerDeUtil
2934import org .apache .spark .internal .Logging
3035import org .apache .spark .rdd .RDD
3136import org .apache .spark .streaming .StreamingContext
32- import org .apache .spark .streaming .api .java .{ JavaInputDStream , JavaStreamingContext }
33- import org .apache .spark .streaming .dstream ._
37+ import org .apache .spark .streaming .api .java .{JavaDStream , JavaInputDStream , JavaStreamingContext }
38+ import org .apache .spark .streaming .dstream .InputDStream
3439
3540/**
3641 * :: Experimental ::
@@ -76,22 +81,21 @@ object KafkaUtils extends Logging {
7681 }
7782
7883 /**
79- * :: Experimental ::
80- * Java constructor for a batch-oriented interface for consuming from Kafka.
81- * Starting and ending offsets are specified in advance,
82- * so that you can control exactly-once semantics.
83- * @param keyClass Class of the keys in the Kafka records
84- * @param valueClass Class of the values in the Kafka records
85- * @param kafkaParams Kafka
86- * <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
87- * configuration parameters</a>. Requires "bootstrap.servers" to be set
88- * with Kafka broker(s) specified in host1:port1,host2:port2 form.
89- * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD
90- * @param locationStrategy In most cases, pass in LocationStrategies.preferConsistent,
91- * see [[LocationStrategies ]] for more details.
92- * @tparam K type of Kafka message key
93- * @tparam V type of Kafka message value
94- */
84+ * :: Experimental ::
85+ * Java constructor for a batch-oriented interface for consuming from Kafka.
86+ * Starting and ending offsets are specified in advance,
87+ * so that you can control exactly-once semantics.
88+ *
89+ * @param kafkaParams Kafka
90+ * <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs">
91+ * configuration parameters</a>. Requires "bootstrap.servers" to be set
92+ * with Kafka broker(s) specified in host1:port1,host2:port2 form.
93+ * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD
94+ * @param locationStrategy In most cases, pass in LocationStrategies.preferConsistent,
95+ * see [[LocationStrategies ]] for more details.
96+ * @tparam K type of Kafka message key
97+ * @tparam V type of Kafka message value
98+ */
9599 @ Experimental
96100 def createRDD [K , V ](
97101 jsc : JavaSparkContext ,
@@ -104,19 +108,20 @@ object KafkaUtils extends Logging {
104108 }
105109
106110 /**
107- * :: Experimental ::
108- * Scala constructor for a DStream where
109- * each given Kafka topic/partition corresponds to an RDD partition.
110- * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number
111- * of messages
112- * per second that each '''partition''' will accept.
113- * @param locationStrategy In most cases, pass in LocationStrategies.preferConsistent,
114- * see [[LocationStrategies ]] for more details.
115- * @param consumerStrategy In most cases, pass in ConsumerStrategies.subscribe,
116- * see [[ConsumerStrategies ]] for more details
117- * @tparam K type of Kafka message key
118- * @tparam V type of Kafka message value
119- */
111+ * :: Experimental ::
112+ * Scala constructor for a DStream where
113+ * each given Kafka topic/partition corresponds to an RDD partition.
114+ * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number
115+ * of messages
116+ * per second that each '''partition''' will accept.
117+ *
118+ * @param locationStrategy In most cases, pass in LocationStrategies.preferConsistent,
119+ * see [[LocationStrategies ]] for more details.
120+ * @param consumerStrategy In most cases, pass in ConsumerStrategies.subscribe,
121+ * see [[ConsumerStrategies ]] for more details
122+ * @tparam K type of Kafka message key
123+ * @tparam V type of Kafka message value
124+ */
120125 @ Experimental
121126 def createDirectStream [K , V ](
122127 ssc : StreamingContext ,
@@ -127,18 +132,17 @@ object KafkaUtils extends Logging {
127132 }
128133
129134 /**
130- * :: Experimental ::
131- * Java constructor for a DStream where
132- * each given Kafka topic/partition corresponds to an RDD partition.
133- * @param keyClass Class of the keys in the Kafka records
134- * @param valueClass Class of the values in the Kafka records
135- * @param locationStrategy In most cases, pass in LocationStrategies.preferConsistent,
136- * see [[LocationStrategies ]] for more details.
137- * @param consumerStrategy In most cases, pass in ConsumerStrategies.subscribe,
138- * see [[ConsumerStrategies ]] for more details
139- * @tparam K type of Kafka message key
140- * @tparam V type of Kafka message value
141- */
135+ * :: Experimental ::
136+ * Java constructor for a DStream where
137+ * each given Kafka topic/partition corresponds to an RDD partition.
138+ *
139+ * @param locationStrategy In most cases, pass in LocationStrategies.preferConsistent,
140+ * see [[LocationStrategies ]] for more details.
141+ * @param consumerStrategy In most cases, pass in ConsumerStrategies.subscribe,
142+ * see [[ConsumerStrategies ]] for more details
143+ * @tparam K type of Kafka message key
144+ * @tparam V type of Kafka message value
145+ */
142146 @ Experimental
143147 def createDirectStream [K , V ](
144148 jssc : JavaStreamingContext ,
@@ -151,8 +155,8 @@ object KafkaUtils extends Logging {
151155 }
152156
153157 /**
154- * Tweak kafka params to prevent issues on executors
155- */
158+ * Tweak kafka params to prevent issues on executors
159+ */
156160 private [kafka09] def fixKafkaParams (kafkaParams : ju.HashMap [String , Object ]): Unit = {
157161 logWarning(s " overriding ${ConsumerConfig .ENABLE_AUTO_COMMIT_CONFIG } to false for executor " )
158162 kafkaParams.put(ConsumerConfig .ENABLE_AUTO_COMMIT_CONFIG , false : java.lang.Boolean )
@@ -177,3 +181,75 @@ object KafkaUtils extends Logging {
177181 }
178182 }
179183}
184+
185+ object KafkaUtilsPythonHelper {
186+ private var initialized = false
187+
188+ def initialize (): Unit = {
189+ SerDeUtil .initialize()
190+ synchronized {
191+ if (! initialized) {
192+ new PythonMessageAndMetadataPickler ().register()
193+ initialized = true
194+ }
195+ }
196+ }
197+
198+ initialize()
199+
200+ def picklerIterator (iter : Iterator [ConsumerRecord [Array [Byte ], Array [Byte ]]]
201+ ): Iterator [Array [Byte ]] = {
202+ new SerDeUtil .AutoBatchedPickler (iter)
203+ }
204+
205+ class PythonMessageAndMetadataPickler extends IObjectPickler {
206+ private val module = " pyspark.streaming.kafka"
207+
208+ def register (): Unit = {
209+ Pickler .registerCustomPickler(classOf [ConsumerRecord [Any , Any ]], this )
210+ Pickler .registerCustomPickler(this .getClass, this )
211+ }
212+
213+ def pickle (obj : Object , out : OutputStream , pickler : Pickler ) {
214+ if (obj == this ) {
215+ out.write(Opcodes .GLOBAL )
216+ out.write(s " $module\n KafkaMessageAndMetadata \n " .getBytes(UTF_8 ))
217+ } else {
218+ pickler.save(this )
219+ val msgAndMetaData = obj.asInstanceOf [ConsumerRecord [Array [Byte ], Array [Byte ]]]
220+ out.write(Opcodes .MARK )
221+ pickler.save(msgAndMetaData.topic)
222+ pickler.save(msgAndMetaData.partition)
223+ pickler.save(msgAndMetaData.offset)
224+ pickler.save(msgAndMetaData.key)
225+ pickler.save(msgAndMetaData.value)
226+ out.write(Opcodes .TUPLE )
227+ out.write(Opcodes .REDUCE )
228+ }
229+ }
230+ }
231+
232+ // def createRDDWithoutMessageHandler(
233+ // jsc: JavaSparkContext,
234+ // kafkaParams: JMap[String, String],
235+ // offsetRanges: JList[OffsetRange],
236+ // leaders: JMap[TopicAndPartition, Broker]): JavaRDD[(Array[Byte], Array[Byte])] = {
237+ // val messageHandler =
238+ // (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => (mmd.key, mmd.message)
239+ // new JavaRDD(createRDD(jsc, kafkaParams, offsetRanges, leaders, messageHandler))
240+ // }
241+
242+ @ Experimental
243+ def createDirectStream (
244+ jssc : JavaStreamingContext ,
245+ locationStrategy : LocationStrategy ,
246+ consumerStrategy : ConsumerStrategy [Array [Byte ], Array [Byte ]]
247+ ): JavaDStream [(Array [Byte ], Array [Byte ])] = {
248+
249+ val dStream = KafkaUtils .createDirectStream[Array [Byte ], Array [Byte ]](
250+ jssc.ssc, locationStrategy, consumerStrategy)
251+ .map(cm => (cm.key, cm.value))
252+
253+ new JavaDStream [(Array [Byte ], Array [Byte ])](dStream)
254+ }
255+ }
0 commit comments