|
16 | 16 | # |
17 | 17 |
|
18 | 18 | from py4j.java_collections import MapConverter |
19 | | -from py4j.java_gateway import java_import, Py4JError |
| 19 | +from py4j.java_gateway import java_import, Py4JError, Py4JJavaError |
20 | 20 |
|
21 | 21 | from pyspark.storagelevel import StorageLevel |
22 | 22 | from pyspark.serializers import PairDeserializer, NoOpSerializer |
@@ -63,20 +63,32 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, |
63 | 63 | jparam = MapConverter().convert(kafkaParams, ssc.sparkContext._gateway._gateway_client) |
64 | 64 | jlevel = ssc._sc._getJavaStorageLevel(storageLevel) |
65 | 65 |
|
66 | | - def getClassByName(name): |
67 | | - return ssc._jvm.org.apache.spark.util.Utils.classForName(name) |
68 | | - |
69 | 66 | try: |
70 | | - array = getClassByName("[B") |
71 | | - decoder = getClassByName("kafka.serializer.DefaultDecoder") |
72 | | - jstream = ssc._jvm.KafkaUtils.createStream(ssc._jssc, array, array, decoder, decoder, |
73 | | - jparam, jtopics, jlevel) |
74 | | - except Py4JError, e: |
| 67 | + helperClass = ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader().loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper") |
| 68 | + helper = helperClass.newInstance() |
| 69 | + jstream = helper.createStream(ssc._jssc, jparam, jtopics, jlevel) |
| 70 | + except Py4JJavaError, e: |
75 | 71 | # TODO: use --jar once it also work on driver |
76 | | - if not e.message or 'call a package' in e.message: |
77 | | - print "No kafka package, please put the assembly jar into classpath:" |
78 | | - print " $ bin/spark-submit --driver-class-path external/kafka-assembly/target/" + \ |
79 | | - "scala-*/spark-streaming-kafka-assembly-*.jar" |
| 72 | + if 'ClassNotFoundException' in str(e.java_exception): |
| 73 | + print """ |
| 74 | +________________________________________________________________________________________________ |
| 75 | +
|
| 76 | + Spark Streaming's Kafka libraries not found in class path. Try one of the following. |
| 77 | +
|
| 78 | + 1. Include the Kafka library and its dependencies with in the |
| 79 | + spark-submit command as |
| 80 | +
|
| 81 | + $ bin/spark-submit --package org.apache.spark:spark-streaming-kafka:%s ... |
| 82 | +
|
| 83 | + 2. Download the JAR of the artifact from Maven Central http://search.maven.org/, |
| 84 | + Group Id = org.apache.spark, Artifact Id = spark-streaming-kafka-assembly, Version = %s. |
| 85 | + Then, innclude the jar in the spark-submit command as |
| 86 | +
|
| 87 | + $ bin/spark-submit --jars <spark-streaming-kafka-assembly.jar> ... |
| 88 | +
|
| 89 | +________________________________________________________________________________________________ |
| 90 | +
|
| 91 | +""" % (ssc.sparkContext.version, ssc.sparkContext.version) |
80 | 92 | raise e |
81 | 93 | ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) |
82 | 94 | stream = DStream(jstream, ssc, ser) |
|
0 commit comments