aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/streaming/kafka.py
diff options
context:
space:
mode:
Diffstat (limited to 'python/pyspark/streaming/kafka.py')
-rw-r--r--python/pyspark/streaming/kafka.py100
1 files changed, 41 insertions, 59 deletions
diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py
index 13f8f9578e..a70b99249d 100644
--- a/python/pyspark/streaming/kafka.py
+++ b/python/pyspark/streaming/kafka.py
@@ -66,18 +66,8 @@ class KafkaUtils(object):
if not isinstance(topics, dict):
raise TypeError("topics should be dict")
jlevel = ssc._sc._getJavaStorageLevel(storageLevel)
-
- try:
- # Use KafkaUtilsPythonHelper to access Scala's KafkaUtils (see SPARK-6027)
- helperClass = ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader()\
- .loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper")
- helper = helperClass.newInstance()
- jstream = helper.createStream(ssc._jssc, kafkaParams, topics, jlevel)
- except Py4JJavaError as e:
- # TODO: use --jar once it also work on driver
- if 'ClassNotFoundException' in str(e.java_exception):
- KafkaUtils._printErrorMsg(ssc.sparkContext)
- raise e
+ helper = KafkaUtils._get_helper(ssc._sc)
+ jstream = helper.createStream(ssc._jssc, kafkaParams, topics, jlevel)
ser = PairDeserializer(NoOpSerializer(), NoOpSerializer())
stream = DStream(jstream, ssc, ser)
return stream.map(lambda k_v: (keyDecoder(k_v[0]), valueDecoder(k_v[1])))
@@ -129,27 +119,20 @@ class KafkaUtils(object):
m._set_value_decoder(valueDecoder)
return messageHandler(m)
- try:
- helperClass = ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \
- .loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper")
- helper = helperClass.newInstance()
-
- jfromOffsets = dict([(k._jTopicAndPartition(helper),
- v) for (k, v) in fromOffsets.items()])
- if messageHandler is None:
- ser = PairDeserializer(NoOpSerializer(), NoOpSerializer())
- func = funcWithoutMessageHandler
- jstream = helper.createDirectStreamWithoutMessageHandler(
- ssc._jssc, kafkaParams, set(topics), jfromOffsets)
- else:
- ser = AutoBatchedSerializer(PickleSerializer())
- func = funcWithMessageHandler
- jstream = helper.createDirectStreamWithMessageHandler(
- ssc._jssc, kafkaParams, set(topics), jfromOffsets)
- except Py4JJavaError as e:
- if 'ClassNotFoundException' in str(e.java_exception):
- KafkaUtils._printErrorMsg(ssc.sparkContext)
- raise e
+ helper = KafkaUtils._get_helper(ssc._sc)
+
+ jfromOffsets = dict([(k._jTopicAndPartition(helper),
+ v) for (k, v) in fromOffsets.items()])
+ if messageHandler is None:
+ ser = PairDeserializer(NoOpSerializer(), NoOpSerializer())
+ func = funcWithoutMessageHandler
+ jstream = helper.createDirectStreamWithoutMessageHandler(
+ ssc._jssc, kafkaParams, set(topics), jfromOffsets)
+ else:
+ ser = AutoBatchedSerializer(PickleSerializer())
+ func = funcWithMessageHandler
+ jstream = helper.createDirectStreamWithMessageHandler(
+ ssc._jssc, kafkaParams, set(topics), jfromOffsets)
stream = DStream(jstream, ssc, ser).map(func)
return KafkaDStream(stream._jdstream, ssc, stream._jrdd_deserializer)
@@ -189,28 +172,35 @@ class KafkaUtils(object):
m._set_value_decoder(valueDecoder)
return messageHandler(m)
+ helper = KafkaUtils._get_helper(sc)
+
+ joffsetRanges = [o._jOffsetRange(helper) for o in offsetRanges]
+ jleaders = dict([(k._jTopicAndPartition(helper),
+ v._jBroker(helper)) for (k, v) in leaders.items()])
+ if messageHandler is None:
+ jrdd = helper.createRDDWithoutMessageHandler(
+ sc._jsc, kafkaParams, joffsetRanges, jleaders)
+ ser = PairDeserializer(NoOpSerializer(), NoOpSerializer())
+ rdd = RDD(jrdd, sc, ser).map(funcWithoutMessageHandler)
+ else:
+ jrdd = helper.createRDDWithMessageHandler(
+ sc._jsc, kafkaParams, joffsetRanges, jleaders)
+ rdd = RDD(jrdd, sc).map(funcWithMessageHandler)
+
+ return KafkaRDD(rdd._jrdd, sc, rdd._jrdd_deserializer)
+
+ @staticmethod
+ def _get_helper(sc):
try:
+ # Use KafkaUtilsPythonHelper to access Scala's KafkaUtils (see SPARK-6027)
helperClass = sc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \
.loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper")
- helper = helperClass.newInstance()
- joffsetRanges = [o._jOffsetRange(helper) for o in offsetRanges]
- jleaders = dict([(k._jTopicAndPartition(helper),
- v._jBroker(helper)) for (k, v) in leaders.items()])
- if messageHandler is None:
- jrdd = helper.createRDDWithoutMessageHandler(
- sc._jsc, kafkaParams, joffsetRanges, jleaders)
- ser = PairDeserializer(NoOpSerializer(), NoOpSerializer())
- rdd = RDD(jrdd, sc, ser).map(funcWithoutMessageHandler)
- else:
- jrdd = helper.createRDDWithMessageHandler(
- sc._jsc, kafkaParams, joffsetRanges, jleaders)
- rdd = RDD(jrdd, sc).map(funcWithMessageHandler)
+ return helperClass.newInstance()
except Py4JJavaError as e:
+ # TODO: use --jar once it also work on driver
if 'ClassNotFoundException' in str(e.java_exception):
KafkaUtils._printErrorMsg(sc)
- raise e
-
- return KafkaRDD(rdd._jrdd, sc, rdd._jrdd_deserializer)
+ raise
@staticmethod
def _printErrorMsg(sc):
@@ -333,16 +323,8 @@ class KafkaRDD(RDD):
Get the OffsetRange of specific KafkaRDD.
:return: A list of OffsetRange
"""
- try:
- helperClass = self.ctx._jvm.java.lang.Thread.currentThread().getContextClassLoader() \
- .loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper")
- helper = helperClass.newInstance()
- joffsetRanges = helper.offsetRangesOfKafkaRDD(self._jrdd.rdd())
- except Py4JJavaError as e:
- if 'ClassNotFoundException' in str(e.java_exception):
- KafkaUtils._printErrorMsg(self.ctx)
- raise e
-
+ helper = KafkaUtils._get_helper(self.ctx)
+ joffsetRanges = helper.offsetRangesOfKafkaRDD(self._jrdd.rdd())
ranges = [OffsetRange(o.topic(), o.partition(), o.fromOffset(), o.untilOffset())
for o in joffsetRanges]
return ranges