[ 
https://issues.apache.org/jira/browse/SPARK-16802?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sylvain Zimmer updated SPARK-16802:
-----------------------------------
    Description: 
Hello!

This is a little similar to 
[SPARK-16740|https://issues.apache.org/jira/browse/SPARK-16740] (should I have 
reopened it?).

I would recommend to give another full review to {{HashedRelation.scala}}, 
particularly the new {{LongToUnsafeRowMap}} code. I've had a few other errors 
that I haven't managed to reproduce so far, as well as what I suspect could be 
memory leaks (I have a query in a loop OOMing after a few iterations despite 
not caching its results).

Here is the script to reproduce the ArrayIndexOutOfBoundsException on the 
current 2.0 branch:

{code}
import os
import random

from pyspark import SparkContext
from pyspark.sql import types as SparkTypes
from pyspark.sql import SQLContext

sc = SparkContext()
sqlc = SQLContext(sc)

schema1 = SparkTypes.StructType([
    SparkTypes.StructField("id1", SparkTypes.LongType(), nullable=True),
    # SparkTypes.StructField("weight1", SparkTypes.FloatType(), nullable=True)
])
schema2 = SparkTypes.StructType([
    SparkTypes.StructField("id2", SparkTypes.LongType(), nullable=True),
    # SparkTypes.StructField("weight2", SparkTypes.LongType(), nullable=True)
])


def randlong():
    return random.randint(-9223372036854775808, 9223372036854775807)

while True:
    l1, l2 = randlong(), randlong()

    # Sample values that crash:
    # l1, l2 = 4661454128115150227, -5543241376386463808

    print "Testing with %s, %s" % (l1, l2)
    data1 = [(l1, ), (l2, )]
    data2 = [(l1, )]

    df1 = sqlc.createDataFrame(sc.parallelize(data1), schema1)
    df2 = sqlc.createDataFrame(sc.parallelize(data2), schema2)

    crash = True
    if crash:
        os.system("rm -rf /tmp/sparkbug")
        df1.write.parquet("/tmp/sparkbug/vertex")
        df2.write.parquet("/tmp/sparkbug/edge")

        df1 = sqlc.read.load("/tmp/sparkbug/vertex")
        df2 = sqlc.read.load("/tmp/sparkbug/edge")

    sqlc.registerDataFrameAsTable(df1, "df1")
    sqlc.registerDataFrameAsTable(df2, "df2")

    result_df = sqlc.sql("""
        SELECT
            df1.id1
        FROM df1
        LEFT OUTER JOIN df2 ON df1.id1 = df2.id2
    """)

    print result_df.collect()
{code}

{code}
java.lang.ArrayIndexOutOfBoundsException: 1728150825
        at 
org.apache.spark.sql.execution.joins.LongToUnsafeRowMap.getValue(HashedRelation.scala:463)
        at 
org.apache.spark.sql.execution.joins.LongHashedRelation.getValue(HashedRelation.scala:762)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown
 Source)
        at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
        at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
        at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.hasNext(SerDeUtil.scala:117)
        at scala.collection.Iterator$class.foreach(Iterator.scala:893)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.foreach(SerDeUtil.scala:112)
        at 
scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
        at 
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
        at 
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
        at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:310)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.to(SerDeUtil.scala:112)
        at 
scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:302)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.toBuffer(SerDeUtil.scala:112)
        at 
scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:289)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.toArray(SerDeUtil.scala:112)
        at 
org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:899)
        at 
org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:899)
        at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1898)
        at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1898)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
        at org.apache.spark.scheduler.Task.run(Task.scala:85)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
16/07/29 20:19:00 WARN TaskSetManager: Lost task 0.0 in stage 17.0 (TID 50, 
localhost): java.lang.ArrayIndexOutOfBoundsException: 1728150825
        at 
org.apache.spark.sql.execution.joins.LongToUnsafeRowMap.getValue(HashedRelation.scala:463)
        at 
org.apache.spark.sql.execution.joins.LongHashedRelation.getValue(HashedRelation.scala:762)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown
 Source)
        at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
        at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
        at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.hasNext(SerDeUtil.scala:117)
        at scala.collection.Iterator$class.foreach(Iterator.scala:893)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.foreach(SerDeUtil.scala:112)
        at 
scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
        at 
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
        at 
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
        at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:310)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.to(SerDeUtil.scala:112)
        at 
scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:302)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.toBuffer(SerDeUtil.scala:112)
        at 
scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:289)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.toArray(SerDeUtil.scala:112)
        at 
org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:899)
        at 
org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:899)
        at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1898)
        at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1898)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
        at org.apache.spark.scheduler.Task.run(Task.scala:85)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
{code}

  was:
Hello!

This is a little similar to 
[SPARK-16740|https://issues.apache.org/jira/browse/SPARK-16740] (should I have 
reopened it?).

I would recommend to give another full review to {{HashedRelation.scala}}, 
particularly the new {{LongToUnsafeRowMap}} code. I've had a few other errors 
that I haven't managed to reproduce so far, as well as what I suspect could be 
memory leaks (I have a query in a loop OOMing after a few iterations despite 
not caching its results).

Here is the script to reproduce the ArrayIndexOutOfBoundsException on the 
current 2.0 branch:

{{code}}
import os
import random

from pyspark import SparkContext
from pyspark.sql import types as SparkTypes
from pyspark.sql import SQLContext

sc = SparkContext()
sqlc = SQLContext(sc)

schema1 = SparkTypes.StructType([
    SparkTypes.StructField("id1", SparkTypes.LongType(), nullable=True),
    # SparkTypes.StructField("weight1", SparkTypes.FloatType(), nullable=True)
])
schema2 = SparkTypes.StructType([
    SparkTypes.StructField("id2", SparkTypes.LongType(), nullable=True),
    # SparkTypes.StructField("weight2", SparkTypes.LongType(), nullable=True)
])


def randlong():
    return random.randint(-9223372036854775808, 9223372036854775807)

while True:
    l1, l2 = randlong(), randlong()

    # Sample values that crash:
    # l1, l2 = 4661454128115150227, -5543241376386463808

    print "Testing with %s, %s" % (l1, l2)
    data1 = [(l1, ), (l2, )]
    data2 = [(l1, )]

    df1 = sqlc.createDataFrame(sc.parallelize(data1), schema1)
    df2 = sqlc.createDataFrame(sc.parallelize(data2), schema2)

    crash = True
    if crash:
        os.system("rm -rf /tmp/sparkbug")
        df1.write.parquet("/tmp/sparkbug/vertex")
        df2.write.parquet("/tmp/sparkbug/edge")

        df1 = sqlc.read.load("/tmp/sparkbug/vertex")
        df2 = sqlc.read.load("/tmp/sparkbug/edge")

    sqlc.registerDataFrameAsTable(df1, "df1")
    sqlc.registerDataFrameAsTable(df2, "df2")

    result_df = sqlc.sql("""
        SELECT
            df1.id1
        FROM df1
        LEFT OUTER JOIN df2 ON df1.id1 = df2.id2
    """)

    print result_df.collect()
{{code}}

{{code}}
java.lang.ArrayIndexOutOfBoundsException: 1728150825
        at 
org.apache.spark.sql.execution.joins.LongToUnsafeRowMap.getValue(HashedRelation.scala:463)
        at 
org.apache.spark.sql.execution.joins.LongHashedRelation.getValue(HashedRelation.scala:762)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown
 Source)
        at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
        at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
        at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.hasNext(SerDeUtil.scala:117)
        at scala.collection.Iterator$class.foreach(Iterator.scala:893)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.foreach(SerDeUtil.scala:112)
        at 
scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
        at 
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
        at 
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
        at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:310)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.to(SerDeUtil.scala:112)
        at 
scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:302)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.toBuffer(SerDeUtil.scala:112)
        at 
scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:289)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.toArray(SerDeUtil.scala:112)
        at 
org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:899)
        at 
org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:899)
        at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1898)
        at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1898)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
        at org.apache.spark.scheduler.Task.run(Task.scala:85)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
16/07/29 20:19:00 WARN TaskSetManager: Lost task 0.0 in stage 17.0 (TID 50, 
localhost): java.lang.ArrayIndexOutOfBoundsException: 1728150825
        at 
org.apache.spark.sql.execution.joins.LongToUnsafeRowMap.getValue(HashedRelation.scala:463)
        at 
org.apache.spark.sql.execution.joins.LongHashedRelation.getValue(HashedRelation.scala:762)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown
 Source)
        at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
        at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
        at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.hasNext(SerDeUtil.scala:117)
        at scala.collection.Iterator$class.foreach(Iterator.scala:893)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.foreach(SerDeUtil.scala:112)
        at 
scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
        at 
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
        at 
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
        at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:310)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.to(SerDeUtil.scala:112)
        at 
scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:302)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.toBuffer(SerDeUtil.scala:112)
        at 
scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:289)
        at 
org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.toArray(SerDeUtil.scala:112)
        at 
org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:899)
        at 
org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:899)
        at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1898)
        at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1898)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
        at org.apache.spark.scheduler.Task.run(Task.scala:85)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
{{code}}



> joins.LongToUnsafeRowMap crashes with ArrayIndexOutOfBoundsException
> --------------------------------------------------------------------
>
>                 Key: SPARK-16802
>                 URL: https://issues.apache.org/jira/browse/SPARK-16802
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.0.0, 2.0.1
>            Reporter: Sylvain Zimmer
>
> Hello!
> This is a little similar to 
> [SPARK-16740|https://issues.apache.org/jira/browse/SPARK-16740] (should I 
> have reopened it?).
> I would recommend to give another full review to {{HashedRelation.scala}}, 
> particularly the new {{LongToUnsafeRowMap}} code. I've had a few other errors 
> that I haven't managed to reproduce so far, as well as what I suspect could 
> be memory leaks (I have a query in a loop OOMing after a few iterations 
> despite not caching its results).
> Here is the script to reproduce the ArrayIndexOutOfBoundsException on the 
> current 2.0 branch:
> {code}
> import os
> import random
> from pyspark import SparkContext
> from pyspark.sql import types as SparkTypes
> from pyspark.sql import SQLContext
> sc = SparkContext()
> sqlc = SQLContext(sc)
> schema1 = SparkTypes.StructType([
>     SparkTypes.StructField("id1", SparkTypes.LongType(), nullable=True),
>     # SparkTypes.StructField("weight1", SparkTypes.FloatType(), nullable=True)
> ])
> schema2 = SparkTypes.StructType([
>     SparkTypes.StructField("id2", SparkTypes.LongType(), nullable=True),
>     # SparkTypes.StructField("weight2", SparkTypes.LongType(), nullable=True)
> ])
> def randlong():
>     return random.randint(-9223372036854775808, 9223372036854775807)
> while True:
>     l1, l2 = randlong(), randlong()
>     # Sample values that crash:
>     # l1, l2 = 4661454128115150227, -5543241376386463808
>     print "Testing with %s, %s" % (l1, l2)
>     data1 = [(l1, ), (l2, )]
>     data2 = [(l1, )]
>     df1 = sqlc.createDataFrame(sc.parallelize(data1), schema1)
>     df2 = sqlc.createDataFrame(sc.parallelize(data2), schema2)
>     crash = True
>     if crash:
>         os.system("rm -rf /tmp/sparkbug")
>         df1.write.parquet("/tmp/sparkbug/vertex")
>         df2.write.parquet("/tmp/sparkbug/edge")
>         df1 = sqlc.read.load("/tmp/sparkbug/vertex")
>         df2 = sqlc.read.load("/tmp/sparkbug/edge")
>     sqlc.registerDataFrameAsTable(df1, "df1")
>     sqlc.registerDataFrameAsTable(df2, "df2")
>     result_df = sqlc.sql("""
>         SELECT
>             df1.id1
>         FROM df1
>         LEFT OUTER JOIN df2 ON df1.id1 = df2.id2
>     """)
>     print result_df.collect()
> {code}
> {code}
> java.lang.ArrayIndexOutOfBoundsException: 1728150825
>       at 
> org.apache.spark.sql.execution.joins.LongToUnsafeRowMap.getValue(HashedRelation.scala:463)
>       at 
> org.apache.spark.sql.execution.joins.LongHashedRelation.getValue(HashedRelation.scala:762)
>       at 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown
>  Source)
>       at 
> org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
>       at 
> org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
>       at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
>       at 
> org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.hasNext(SerDeUtil.scala:117)
>       at scala.collection.Iterator$class.foreach(Iterator.scala:893)
>       at 
> org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.foreach(SerDeUtil.scala:112)
>       at 
> scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
>       at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
>       at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
>       at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:310)
>       at 
> org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.to(SerDeUtil.scala:112)
>       at 
> scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:302)
>       at 
> org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.toBuffer(SerDeUtil.scala:112)
>       at 
> scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:289)
>       at 
> org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.toArray(SerDeUtil.scala:112)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:899)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:899)
>       at 
> org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1898)
>       at 
> org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1898)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
>       at org.apache.spark.scheduler.Task.run(Task.scala:85)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> 16/07/29 20:19:00 WARN TaskSetManager: Lost task 0.0 in stage 17.0 (TID 50, 
> localhost): java.lang.ArrayIndexOutOfBoundsException: 1728150825
>       at 
> org.apache.spark.sql.execution.joins.LongToUnsafeRowMap.getValue(HashedRelation.scala:463)
>       at 
> org.apache.spark.sql.execution.joins.LongHashedRelation.getValue(HashedRelation.scala:762)
>       at 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown
>  Source)
>       at 
> org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
>       at 
> org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
>       at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
>       at 
> org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.hasNext(SerDeUtil.scala:117)
>       at scala.collection.Iterator$class.foreach(Iterator.scala:893)
>       at 
> org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.foreach(SerDeUtil.scala:112)
>       at 
> scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
>       at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
>       at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
>       at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:310)
>       at 
> org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.to(SerDeUtil.scala:112)
>       at 
> scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:302)
>       at 
> org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.toBuffer(SerDeUtil.scala:112)
>       at 
> scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:289)
>       at 
> org.apache.spark.api.python.SerDeUtil$AutoBatchedPickler.toArray(SerDeUtil.scala:112)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:899)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:899)
>       at 
> org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1898)
>       at 
> org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1898)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
>       at org.apache.spark.scheduler.Task.run(Task.scala:85)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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

Reply via email to