@@ -36,16 +36,6 @@ def utf8_decoder(s):
3636 return s .decode ('utf-8' )
3737
3838
39- def default_message_handler (s ):
40- """
41- Function for translating each message and metadata into the desired type
42-
43- :param s: A KafkaMessageAndMetadata object includes message and metadata
44- :return: A tuple of Kafka key and message
45- """
46- return s and (s .key , s .message )
47-
48-
4939class KafkaUtils (object ):
5040
5141 @staticmethod
@@ -95,7 +85,7 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams=None,
9585 @staticmethod
9686 def createDirectStream (ssc , topics , kafkaParams , fromOffsets = None ,
9787 keyDecoder = utf8_decoder , valueDecoder = utf8_decoder ,
98- messageHandler = default_message_handler ):
88+ messageHandler = None ):
9989 """
10090 .. note:: Experimental
10191
@@ -120,6 +110,8 @@ def createDirectStream(ssc, topics, kafkaParams, fromOffsets=None,
120110 point of the stream.
121111 :param keyDecoder: A function used to decode key (default is utf8_decoder).
122112 :param valueDecoder: A function used to decode value (default is utf8_decoder).
113+ :param messageHandler: A function used to convert KafkaMessageAndMetadata. You can assess
114+ meta using messageHandler (default is None).
123115 :return: A DStream object
124116 """
125117 if fromOffsets is None :
@@ -129,32 +121,43 @@ def createDirectStream(ssc, topics, kafkaParams, fromOffsets=None,
129121 if not isinstance (kafkaParams , dict ):
130122 raise TypeError ("kafkaParams should be dict" )
131123
124+ def funcWithoutMessageHandler (k_v ):
125+ return (keyDecoder (k_v [0 ]), valueDecoder (k_v [1 ]))
126+
127+ def funcWithMessageHandler (m ):
128+ m ._set_key_decoder (keyDecoder )
129+ m ._set_value_decoder (valueDecoder )
130+ return messageHandler (m )
131+
132132 try :
133133 helperClass = ssc ._jvm .java .lang .Thread .currentThread ().getContextClassLoader () \
134134 .loadClass ("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper" )
135135 helper = helperClass .newInstance ()
136136
137137 jfromOffsets = dict ([(k ._jTopicAndPartition (helper ),
138138 v ) for (k , v ) in fromOffsets .items ()])
139- jstream = helper .createDirectStream (ssc ._jssc , kafkaParams , set (topics ), jfromOffsets )
139+ if messageHandler is None :
140+ ser = PairDeserializer (NoOpSerializer (), NoOpSerializer ())
141+ func = funcWithoutMessageHandler
142+ jstream = helper .createDirectStreamWithoutMessageHandler (
143+ ssc ._jssc , kafkaParams , set (topics ), jfromOffsets )
144+ else :
145+ ser = AutoBatchedSerializer (PickleSerializer ())
146+ func = funcWithMessageHandler
147+ jstream = helper .createDirectStreamWithMessageHandler (
148+ ssc ._jssc , kafkaParams , set (topics ), jfromOffsets )
140149 except Py4JJavaError as e :
141150 if 'ClassNotFoundException' in str (e .java_exception ):
142151 KafkaUtils ._printErrorMsg (ssc .sparkContext )
143152 raise e
144153
145- def func (m ):
146- m ._set_key_decoder (keyDecoder )
147- m ._set_value_decoder (valueDecoder )
148- return messageHandler (m )
149-
150- ser = AutoBatchedSerializer (PickleSerializer ())
151154 stream = DStream (jstream , ssc , ser ).map (func )
152155 return KafkaDStream (stream ._jdstream , ssc , stream ._jrdd_deserializer )
153156
154157 @staticmethod
155158 def createRDD (sc , kafkaParams , offsetRanges , leaders = None ,
156159 keyDecoder = utf8_decoder , valueDecoder = utf8_decoder ,
157- messageHandler = default_message_handler ):
160+ messageHandler = None ):
158161 """
159162 .. note:: Experimental
160163
@@ -167,6 +170,8 @@ def createRDD(sc, kafkaParams, offsetRanges, leaders=None,
167170 map, in which case leaders will be looked up on the driver.
168171 :param keyDecoder: A function used to decode key (default is utf8_decoder)
169172 :param valueDecoder: A function used to decode value (default is utf8_decoder)
173+ :param messageHandler: A function used to convert KafkaMessageAndMetadata. You can assess
174+ meta using messageHandler (default is None).
170175 :return: A RDD object
171176 """
172177 if leaders is None :
@@ -176,25 +181,35 @@ def createRDD(sc, kafkaParams, offsetRanges, leaders=None,
176181 if not isinstance (offsetRanges , list ):
177182 raise TypeError ("offsetRanges should be list" )
178183
184+ def funcWithoutMessageHandler (k_v ):
185+ return (keyDecoder (k_v [0 ]), valueDecoder (k_v [1 ]))
186+
187+ def funcWithMessageHandler (m ):
188+ m ._set_key_decoder (keyDecoder )
189+ m ._set_value_decoder (valueDecoder )
190+ return messageHandler (m )
191+
179192 try :
180193 helperClass = sc ._jvm .java .lang .Thread .currentThread ().getContextClassLoader () \
181194 .loadClass ("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper" )
182195 helper = helperClass .newInstance ()
183196 joffsetRanges = [o ._jOffsetRange (helper ) for o in offsetRanges ]
184197 jleaders = dict ([(k ._jTopicAndPartition (helper ),
185198 v ._jBroker (helper )) for (k , v ) in leaders .items ()])
186- jrdd = helper .createRDD (sc ._jsc , kafkaParams , joffsetRanges , jleaders )
199+ if messageHandler is None :
200+ jrdd = helper .createRDDWithoutMessageHandler (
201+ sc ._jsc , kafkaParams , joffsetRanges , jleaders )
202+ ser = PairDeserializer (NoOpSerializer (), NoOpSerializer ())
203+ rdd = RDD (jrdd , sc , ser ).map (funcWithoutMessageHandler )
204+ else :
205+ jrdd = helper .createRDDWithMessageHandler (
206+ sc ._jsc , kafkaParams , joffsetRanges , jleaders )
207+ rdd = RDD (jrdd , sc ).map (funcWithMessageHandler )
187208 except Py4JJavaError as e :
188209 if 'ClassNotFoundException' in str (e .java_exception ):
189210 KafkaUtils ._printErrorMsg (sc )
190211 raise e
191212
192- def func (m ):
193- m ._set_key_decoder (keyDecoder )
194- m ._set_value_decoder (valueDecoder )
195- return messageHandler (m )
196-
197- rdd = RDD (jrdd , sc ).map (func )
198213 return KafkaRDD (rdd ._jrdd , sc , rdd ._jrdd_deserializer )
199214
200215 @staticmethod
0 commit comments