Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3715#discussion_r23978974
  
    --- Diff: python/pyspark/streaming/kafka.py ---
    @@ -0,0 +1,83 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from py4j.java_collections import MapConverter
    +from py4j.java_gateway import java_import, Py4JError
    +
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.serializers import PairDeserializer, NoOpSerializer
    +from pyspark.streaming import DStream
    +
    +__all__ = ['KafkaUtils', 'utf8_decoder']
    +
    +
    +def utf8_decoder(s):
    +    """ Decode the unicode as UTF-8 """
    +    return s and s.decode('utf-8')
    +
    +
    +class KafkaUtils(object):
    +
    +    @staticmethod
    +    def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={},
    +                     storageLevel=StorageLevel.MEMORY_AND_DISK_SER_2,
    +                     keyDecoder=utf8_decoder, valueDecoder=utf8_decoder):
    +        """
    +        Create an input stream that pulls messages from a Kafka Broker.
    +
    +        :param ssc:  StreamingContext object
    +        :param zkQuorum:  Zookeeper quorum 
(hostname:port,hostname:port,..).
    +        :param groupId:  The group id for this consumer.
    +        :param topics:  Dict of (topic_name -> numPartitions) to consume.
    +                        Each partition is consumed in its own thread.
    +        :param kafkaParams: Additional params for Kafka
    +        :param storageLevel:  RDD storage level.
    +        :param keyDecoder:  A function used to decode key (default is 
utf8_decoder)
    +        :param valueDecoder:  A function used to decode value (default is 
utf8_decoder)
    +        :return: A DStream object
    +        """
    +        java_import(ssc._jvm, 
"org.apache.spark.streaming.kafka.KafkaUtils")
    +
    +        kafkaParams.update({
    +            "zookeeper.connect": zkQuorum,
    +            "group.id": groupId,
    +            "zookeeper.connection.timeout.ms": "10000",
    +        })
    +        if not isinstance(topics, dict):
    +            raise TypeError("topics should be dict")
    +        jtopics = MapConverter().convert(topics, 
ssc.sparkContext._gateway._gateway_client)
    +        jparam = MapConverter().convert(kafkaParams, 
ssc.sparkContext._gateway._gateway_client)
    +        jlevel = ssc._sc._getJavaStorageLevel(storageLevel)
    +
    +        def getClassByName(name):
    +            return ssc._jvm.org.apache.spark.util.Utils.classForName(name)
    +
    +        try:
    +            array = getClassByName("[B")
    +            decoder = getClassByName("kafka.serializer.DefaultDecoder")
    +            jstream = ssc._jvm.KafkaUtils.createStream(ssc._jssc, array, 
array, decoder, decoder,
    +                                                       jparam, jtopics, 
jlevel)
    +        except Py4JError, e:
    +            # TODO: use --jar once it also work on driver
    +            if not e.message or 'call a package' in e.message:
    +                print "No kafka package, please put the assembly jar into 
classpath:"
    +                print " $ bin/spark-submit --driver-class-path 
external/kafka-assembly/target/" + \
    +                      "scala-*/spark-streaming-kafka-assembly-*.jar"
    +            raise e
    --- End diff --
    
    The message that gets printed here is quite scary. 
    ```
    
    2015-02-02 18:31:31.950 java[76691:5f03] Unable to load realm info from 
SCDynamicStore
    No kafka package, please put the assembly jar into classpath:
     $ bin/spark-submit --driver-class-path 
external/kafka-assembly/target/scala-*/spark-streaming-kafka-assembly-*.jar
    Traceback (most recent call last):
      File 
"/Users/tdas/Projects/Spark/spark/examples/src/main/python/streaming/kafka_wordcount.py",
 line 46, in <module>
        kvs = KafkaUtils.createStream(ssc, zkQuorum, 
"spark-streaming-consumer", {topic: 1})
      File 
"/Users/tdas/Projects/Spark/spark/python/pyspark/streaming/kafka.py", line 80, 
in createStream
        raise e
    py4j.protocol.Py4JJavaError: An error occurred while calling 
z:org.apache.spark.util.Utils.classForName.
    : java.lang.ClassNotFoundException: kafka.serializer.DefaultDecoder
        at java.net.URLClassLoader$1.run(URLClassLoader.java:366)
        at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
        at java.security.AccessController.doPrivileged(Native Method)
        at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
        at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
        at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
        at java.lang.Class.forName0(Native Method)
        at java.lang.Class.forName(Class.java:270)
        at org.apache.spark.util.Utils$.classForName(Utils.scala:153)
        at org.apache.spark.util.Utils.classForName(Utils.scala)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:606)
        at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:231)
        at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:379)
        at py4j.Gateway.invoke(Gateway.java:259)
        at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:133)
        at py4j.commands.CallCommand.execute(CallCommand.java:79)
        at py4j.GatewayConnection.run(GatewayConnection.java:207)
        at java.lang.Thread.run(Thread.java:744)
    
    ```
    
    Its easy to miss the real message. Is it possible to quit in a such that 
the whole stack trace does not get printed. Rather it gracefully quits after 
printing this message? Perhaps a quit? @JoshRosen Is that good idea. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to