You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Shawn Guo (JIRA)" <ji...@apache.org> on 2014/08/30 05:34:52 UTC

[jira] [Created] (SPARK-3321) Defining a class within python main script

Shawn Guo created SPARK-3321:
--------------------------------

             Summary: Defining a class within python main script
                 Key: SPARK-3321
                 URL: https://issues.apache.org/jira/browse/SPARK-3321
             Project: Spark
          Issue Type: Bug
          Components: PySpark
    Affects Versions: 1.0.1
         Environment: Python version 2.6.6
Spark version version 1.0.1
jdk1.6.0_43

            Reporter: Shawn Guo
            Priority: Blocker


*leftOuterJoin(self, other, numPartitions=None)*
Perform a left outer join of self and other.
For each element (k, v) in self, the resulting RDD will either contain all pairs (k, (v, w)) for w in other, or the pair (k, (v, None)) if no elements in other have key k.

*Background*: leftOuterJoin will produce None element in result dataset.
I define a new class 'Null' in the main script to replace all python native None to new 'Null' object. 'Null' object overload the [] operator.

{code:title=Class Null|borderStyle=solid}
class Null(object):
        def __getitem__(self,key): return None;
        def __getstate__(self): pass;
        def __setstate__(self, dict): pass;
def convert_to_null(x):
        return Null() if x is None else x

X = A.leftOuterJoin(B)
X.mapValues(lambda line: (line[0],convert_to_null(line[1]))
{code}

The code seems running good in pyspark console, however spark-submit failed with below error messages:
/spark-1.0.1-bin-hadoop1/bin/spark-submit --master local[2] /tmp/python_test.py

{noformat}
  File "/data/work/spark-1.0.1-bin-hadoop1/python/pyspark/worker.py", line 77, in main
    serializer.dump_stream(func(split_index, iterator), outfile)
  File "/data/work/spark-1.0.1-bin-hadoop1/python/pyspark/serializers.py", line 191, in dump_stream
    self.serializer.dump_stream(self._batched(iterator), stream)
  File "/data/work/spark-1.0.1-bin-hadoop1/python/pyspark/serializers.py", line 124, in dump_stream
    self._write_with_length(obj, stream)
  File "/data/work/spark-1.0.1-bin-hadoop1/python/pyspark/serializers.py", line 134, in _write_with_length
    serialized = self.dumps(obj)
  File "/data/work/spark-1.0.1-bin-hadoop1/python/pyspark/serializers.py", line 279, in dumps
    def dumps(self, obj): return cPickle.dumps(obj, 2)
PicklingError: Can't pickle <class '__main__.Null'>: attribute lookup __main__.Null failed

        org.apache.spark.api.python.PythonRDD$$anon$1.read(PythonRDD.scala:115)
        org.apache.spark.api.python.PythonRDD$$anon$1.<init>(PythonRDD.scala:145)
        org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:78)
        org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262)
        org.apache.spark.rdd.RDD.iterator(RDD.scala:229)
        org.apache.spark.rdd.UnionPartition.iterator(UnionRDD.scala:33)
        org.apache.spark.rdd.UnionRDD.compute(UnionRDD.scala:74)
        org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262)
        org.apache.spark.rdd.RDD.iterator(RDD.scala:229)
        org.apache.spark.api.python.PythonRDD$WriterThread$$anonfun$run$1.apply$mcV$sp(PythonRDD.scala:200)
        org.apache.spark.api.python.PythonRDD$WriterThread$$anonfun$run$1.apply(PythonRDD.scala:175)
        org.apache.spark.api.python.PythonRDD$WriterThread$$anonfun$run$1.apply(PythonRDD.scala:175)
        org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1160)
        org.apache.spark.api.python.PythonRDD$WriterThread.run(PythonRDD.scala:174)
Driver stacktrace:
        at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1044)
        at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1028)
        at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1026)
        at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
        at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
        at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1026)
        at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:634)
        at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:634)
        at scala.Option.foreach(Option.scala:236)
        at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:634)
        at org.apache.spark.scheduler.DAGSchedulerEventProcessActor$$anonfun$receive$2.applyOrElse(DAGScheduler.scala:1229)
        at akka.actor.ActorCell.receiveMessage(ActorCell.scala:498)
        at akka.actor.ActorCell.invoke(ActorCell.scala:456)
        at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:237)
        at akka.dispatch.Mailbox.run(Mailbox.scala:219)
        at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386)
        at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
        at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
        at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
        at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
{noformat}

I do google search a lot and fount it may caused by pickling problem in python pickle module. Please refer to the link below:
[Python: defining a class and pickling an instance in the same file|http://stefaanlippens.net/pickleproblem]

One workaround is to define the class in another module seperately, and import it in main script, meanwhile add .py file to be distributed with main script by --py-files parameter.
/spark-1.0.1-bin-hadoop1/bin/spark-submit   *--py-files Null.py*

However I'm thinking if it is possible to define the new class in main script by improving the code?







--
This message was sent by Atlassian JIRA
(v6.2#6252)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org