[ https://issues.apache.org/jira/browse/SPARK-27100?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16863375#comment-16863375 ]
Parth Chandra commented on SPARK-27100: --------------------------------------- The stack overflow is due to serialization of a ShuffleMapTask (see attached file with complete stack) [^SPARK-27100-Overflow.txt] . ShuffleMapTask.partition is a FilePartition and FilePartition.files is a Stream which is essentially a linked list. It is therefore serialized recursively. If the number of files in each partition is, say, 10000 files, recursing into a linked list of length 10000 causes a stack overflow. This is a general problem with serialization of Scala streams (and other collections that are lazily initialized) that is fixed in 2.13 (https://github.com/scala/scala/pull/6676). The problem is only in Bucketed partitions. The corresponding implementation for non Bucketed partitions uses a StreamBuffer. Partial expansion of ShuffleMapTask just before the stack overflow - {code:java} obj = \{ShuffleMapTask@16639} Method threw 'java.lang.StackOverflowError' exception. Cannot evaluate org.apache.spark.scheduler.ShuffleMapTask.toString() taskBinary = \{TorrentBroadcast@17216} Method threw 'java.lang.StackOverflowError' exception. Cannot evaluate org.apache.spark.broadcast.TorrentBroadcast.toString() partition = \{FilePartition@17217} Method threw 'java.lang.StackOverflowError' exception. Cannot evaluate org.apache.spark.sql.execution.datasources.FilePartition.toString() index = 0 files = \{Stream$Cons@17244} Method threw 'java.lang.StackOverflowError' exception. Cannot evaluate scala.collection.immutable.Stream$Cons.toString() hd = \{PartitionedFile@17246} Method threw 'java.lang.StackOverflowError' exception. Cannot evaluate org.apache.spark.sql.execution.datasources.PartitionedFile.toString() partitionValues = \{GenericInternalRow@17259} Method threw 'java.lang.StackOverflowError' exception. Cannot evaluate org.apache.spark.sql.catalyst.expressions.GenericInternalRow.toString() filePath = "hdfs://path/a.db/master/version=1/rangeid=000/part-00039-295e3ac1-760c-482e-8640-5e5d1539c2c9_00000.c000.gz.parquet" start = 0 length = 225781388 locations = \{String[3]@17261} tlVal = \{Stream$Cons@16687} Method threw 'java.lang.StackOverflowError' exception. Cannot evaluate scala.collection.immutable.Stream$Cons.toString() hd = \{PartitionedFile@17249} Method threw 'java.lang.StackOverflowError' exception. Cannot evaluate org.apache.spark.sql.execution.datasources.PartitionedFile.toString() partitionValues = \{GenericInternalRow@17255} Method threw 'java.lang.StackOverflowError' exception. Cannot evaluate org.apache.spark.sql.catalyst.expressions.GenericInternalRow.toString() filePath = "hdfs://path/a.db/master/version=1/rangeid=001/part-00061-0346437e-7f8f-44ac-8739-94d1ee285c0b_00000.c000.gz.parquet" start = 0 length = 431239612 locations = \{String[3]@17257} tlVal = \{Stream$Cons@16812} Method threw 'java.lang.StackOverflowError' exception. Cannot evaluate scala.collection.immutable.Stream$Cons.toString() hd = \{PartitionedFile@17264} Method threw 'java.lang.StackOverflowError' exception. Cannot evaluate org.apache.spark.sql.execution.datasources.PartitionedFile.toString() partitionValues = \{GenericInternalRow@17268} Method threw 'java.lang.StackOverflowError' exception. Cannot evaluate org.apache.spark.sql.catalyst.expressions.GenericInternalRow.toString() filePath = "hdfs://path/a.db/master/version=1/rangeid=002/part-00058-b3a99b18-140e-43ed-838e-276eaa45a5f3_00000.c000.gz.parquet" start = 0 length = 219930113 locations = \{String[3]@17270} tlVal = \{Stream$Cons@17265} Method threw 'java.lang.StackOverflowError' exception. Cannot evaluate scala.collection.immutable.Stream$Cons.toString() hd = \{PartitionedFile@17273} Method threw 'java.lang.StackOverflowError' exception. Cannot evaluate org.apache.spark.sql.execution.datasources.PartitionedFile.toString() partitionValues = \{GenericInternalRow@17277} Method threw 'java.lang.StackOverflowError' exception. Cannot evaluate org.apache.spark.sql.catalyst.expressions.GenericInternalRow.toString() filePath = "hdfs://path/a.db/master/version=1/rangeid=003/part-00051-58be3faa-0611-49de-8546-1656b6086934_00000.c000.gz.parquet" start = 0 length = 219503110 locations = \{String[3]@17279} tlVal = \{Stream$Cons@17274} Method threw 'java.lang.StackOverflowError' exception. Cannot evaluate scala.collection.immutable.Stream$Cons.toString() tlGen = null tlGen = null tlGen = null tlGen = null {code} > dag-scheduler-event-loop" java.lang.StackOverflowError > ------------------------------------------------------ > > Key: SPARK-27100 > URL: https://issues.apache.org/jira/browse/SPARK-27100 > Project: Spark > Issue Type: Bug > Components: MLlib > Affects Versions: 2.1.3, 2.3.3 > Reporter: KaiXu > Priority: Major > Attachments: SPARK-27100-Overflow.txt, stderr > > > ALS in Spark MLlib causes StackOverflow: > /opt/sparkml/spark213/bin/spark-submit --properties-file > /opt/HiBench/report/als/spark/conf/sparkbench/spark.conf --class > com.intel.hibench.sparkbench.ml.ALSExample --master yarn-client > --num-executors 3 --executor-memory 322g > /opt/HiBench/sparkbench/assembly/target/sparkbench-assembly-7.1-SNAPSHOT-dist.jar > --numUsers 40000 --numProducts 60000 --rank 100 --numRecommends 20 > --numIterations 100 --kryo false --implicitPrefs true --numProductBlocks -1 > --numUserBlocks -1 --lambda 1.0 hdfs://bdw-slave20:8020/HiBench/ALS/Input > > Exception in thread "dag-scheduler-event-loop" java.lang.StackOverflowError > at > java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1534) > at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1509) > at > java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432) > at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178) > at > java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1548) > at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1509) > at > java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432) > at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178) > at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:348) > at > scala.collection.immutable.List$SerializationProxy.writeObject(List.scala:468) > at sun.reflect.GeneratedMethodAccessor27.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > at java.io.ObjectStreamClass.invokeWriteObject(ObjectStreamClass.java:1028) > at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1496) > at > java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432) > at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178) > at > java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1548) > at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1509) > at > java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432) > at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178) > at > java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1548) > at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1509) > at > java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432) > at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178) > at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:348) > at > scala.collection.immutable.List$SerializationProxy.writeObject(List.scala:468) > at sun.reflect.GeneratedMethodAccessor27.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > at java.io.ObjectStreamClass.invokeWriteObject(ObjectStreamClass.java:1028) > at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1496) > at > java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432) > at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178) > at > java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1548) > at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1509) > at > java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432) > at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178) > at > java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1548) > at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1509) -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org