Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-5973

zip two rdd with AutoBatchedSerializer will fail

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Fixed
    • 1.2.1, 1.3.0
    • 1.2.2, 1.3.0
    • PySpark
    • None

    Description

      zip two rdd with AutoBatchedSerializer will fail, this bug was introduced by SPARK-4841

      >> a.zip(b).count()
      15/02/24 12:11:56 ERROR PythonRDD: Python worker exited unexpectedly (crashed)
      org.apache.spark.api.python.PythonException: Traceback (most recent call last):
        File "/Users/davies/work/spark/python/pyspark/worker.py", line 101, in main
          process()
        File "/Users/davies/work/spark/python/pyspark/worker.py", line 96, in process
          serializer.dump_stream(func(split_index, iterator), outfile)
        File "/Users/davies/work/spark/python/pyspark/rdd.py", line 2249, in pipeline_func
          return func(split, prev_func(split, iterator))
        File "/Users/davies/work/spark/python/pyspark/rdd.py", line 2249, in pipeline_func
          return func(split, prev_func(split, iterator))
        File "/Users/davies/work/spark/python/pyspark/rdd.py", line 270, in func
          return f(iterator)
        File "/Users/davies/work/spark/python/pyspark/rdd.py", line 933, in <lambda>
          return self.mapPartitions(lambda i: [sum(1 for _ in i)]).sum()
        File "/Users/davies/work/spark/python/pyspark/rdd.py", line 933, in <genexpr>
          return self.mapPartitions(lambda i: [sum(1 for _ in i)]).sum()
        File "/Users/davies/work/spark/python/pyspark/serializers.py", line 306, in load_stream
          " in pair: (%d, %d)" % (len(keys), len(vals)))
      ValueError: Can not deserialize RDD with different number of items in pair: (123, 64)
      

      Attachments

        Issue Links

          Activity

            People

              davies Davies Liu
              davies Davies Liu
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: