You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@spark.apache.org by Koert Kuipers <ko...@tresata.com> on 2016/04/28 20:35:06 UTC

spark 2 segfault

i tried for the first time to run our own in-house unit tests on spark 2,
and i get the error below.
has anyone seen this?

it is reproducible. i tried latest java 7 and it is still there.

# A fatal error has been detected by the Java Runtime Environment:
#
#  SIGSEGV (0xb) at pc=0x00007f7c3a4b1f54, pid=21939, tid=140171011417856
#
# JRE version: Java(TM) SE Runtime Environment (7.0_75-b13) (build
1.7.0_75-b13)
# Java VM: Java HotSpot(TM) 64-Bit Server VM (24.75-b04 mixed mode
linux-amd64 compressed oops)
# Problematic frame:
# V  [libjvm.so+0x747f54]  _Copy_arrayof_conjoint_jlongs+0x44

more info:

Stack: [0x00007f7c1b47e000,0x00007f7c1b57f000],  sp=0x00007f7c1b57a9a8,
free space=1010k
Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native
code)
V  [libjvm.so+0x747f54]  _Copy_arrayof_conjoint_jlongs+0x44
j  sun.misc.Unsafe.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V+0
j
org.apache.spark.unsafe.Platform.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V+34
j  org.apache.spark.unsafe.types.UTF8String.getBytes()[B+76
j  org.apache.spark.unsafe.types.UTF8String.toString()Ljava/lang/String;+5
j
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificSafeProjection.apply(Ljava/lang/Object;)Ljava/lang/Object;+876
j
org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(Lorg/apache/spark/sql/catalyst/InternalRow;)Ljava/lang/Object;+5
j
org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Lorg/apache/spark/sql/catalyst/InternalRow;)Ljava/lang/Object;+11
j
org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Ljava/lang/Object;)Ljava/lang/Object;+5
J 13277 C2
scala.collection.mutable.ArrayOps$ofRef.map(Lscala/Function1;Lscala/collection/generic/CanBuildFrom;)Ljava/lang/Object;
(7 bytes) @ 0x00007f7c25eeae08 [0x00007f7c25eead40+0xc8]
j
org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply()Ljava/lang/Object;+43
j
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(Lorg/apache/spark/sql/SparkSession;Lorg/apache/spark/sql/execution/QueryExecution;Lscala/Function0;)Ljava/lang/Object;+106
j
org.apache.spark.sql.Dataset.withNewExecutionId(Lscala/Function0;)Ljava/lang/Object;+12
j  org.apache.spark.sql.Dataset.org
$apache$spark$sql$Dataset$$execute$1()Ljava/lang/Object;+9
j
org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Lorg/apache/spark/sql/Dataset;)Ljava/lang/Object;+4
j
org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Ljava/lang/Object;)Ljava/lang/Object;+5
j
org.apache.spark.sql.Dataset.withCallback(Ljava/lang/String;Lorg/apache/spark/sql/Dataset;Lscala/Function1;)Ljava/lang/Object;+25
j  org.apache.spark.sql.Dataset.org
$apache$spark$sql$Dataset$$collect(Z)Ljava/lang/Object;+20
j  org.apache.spark.sql.Dataset.collect()Ljava/lang/Object;+2

Re: spark 2 segfault

Posted by Ted Yu <yu...@gmail.com>.
I plan to.

I am not that familiar with all the parts involved though :-)

On Mon, May 2, 2016 at 9:42 AM, Reynold Xin <rx...@databricks.com> wrote:

> Definitely looks like a bug.
>
> Ted - are you looking at this?
>
>
> On Mon, May 2, 2016 at 7:15 AM, Koert Kuipers <ko...@tresata.com> wrote:
>
>> Created issue:
>> https://issues.apache.org/jira/browse/SPARK-15062
>>
>> On Mon, May 2, 2016 at 6:48 AM, Ted Yu <yu...@gmail.com> wrote:
>>
>>> I tried the same statement using Spark 1.6.1
>>> There was no error with default memory setting.
>>>
>>> Suggest logging a bug.
>>>
>>> On May 1, 2016, at 9:22 PM, Koert Kuipers <ko...@tresata.com> wrote:
>>>
>>> Yeah I got that too, then I increased heap for tests to 8G to get error
>>> I showed earlier.
>>> On May 2, 2016 12:09 AM, "Ted Yu" <yu...@gmail.com> wrote:
>>>
>>>> Using commit hash 90787de864b58a1079c23e6581381ca8ffe7685f and
>>>> Java 1.7.0_67 , I got:
>>>>
>>>> scala> val dfComplicated = sc.parallelize(List((Map("1" -> "a"),
>>>> List("b", "c")), (Map("2" -> "b"), List("d", "e")))).toDF
>>>> ...
>>>> dfComplicated: org.apache.spark.sql.DataFrame = [_1:
>>>> map<string,string>, _2: array<string>]
>>>>
>>>> scala> dfComplicated.show
>>>> java.lang.OutOfMemoryError: Java heap space
>>>>   at
>>>> org.apache.spark.unsafe.types.UTF8String.getBytes(UTF8String.java:229)
>>>>   at
>>>> org.apache.spark.unsafe.types.UTF8String.toString(UTF8String.java:821)
>>>>   at
>>>> org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificSafeProjection.apply(Unknown
>>>> Source)
>>>>   at
>>>> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:241)
>>>>   at
>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
>>>>   at
>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
>>>>   at
>>>> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>>>>   at
>>>> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>>>>   at
>>>> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
>>>>   at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
>>>>   at
>>>> scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
>>>>   at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186)
>>>>   at
>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2121)
>>>>   at
>>>> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:54)
>>>>   at org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2408)
>>>>   at org.apache.spark.sql.Dataset.org
>>>> $apache$spark$sql$Dataset$$execute$1(Dataset.scala:2120)
>>>>   at org.apache.spark.sql.Dataset.org
>>>> $apache$spark$sql$Dataset$$collect(Dataset.scala:2127)
>>>>   at
>>>> org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:1861)
>>>>   at
>>>> org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:1860)
>>>>   at org.apache.spark.sql.Dataset.withTypedCallback(Dataset.scala:2438)
>>>>   at org.apache.spark.sql.Dataset.head(Dataset.scala:1860)
>>>>   at org.apache.spark.sql.Dataset.take(Dataset.scala:2077)
>>>>   at org.apache.spark.sql.Dataset.showString(Dataset.scala:238)
>>>>   at org.apache.spark.sql.Dataset.show(Dataset.scala:529)
>>>>   at org.apache.spark.sql.Dataset.show(Dataset.scala:489)
>>>>   at org.apache.spark.sql.Dataset.show(Dataset.scala:498)
>>>>   ... 6 elided
>>>>
>>>> scala>
>>>>
>>>> On Sun, May 1, 2016 at 8:34 PM, Koert Kuipers <ko...@tresata.com>
>>>> wrote:
>>>>
>>>>> by removing dependencies it turns into a different error, see below.
>>>>> the test is simply writing a DataFrame out to file and reading it back
>>>>> in. i see the error for all data sources (json, parquet, etc.).
>>>>>
>>>>> this is the data that i write out and read back in:
>>>>> val dfComplicated = sc.parallelize(List((Map("1" -> "a"), List("b",
>>>>> "c")), (Map("2" -> "b"), List("d", "e")))).toDF
>>>>>
>>>>>
>>>>> [info]   java.lang.RuntimeException: Error while decoding:
>>>>> java.lang.NegativeArraySizeException
>>>>> [info] createexternalrow(if (isnull(input[0, map<string,string>]))
>>>>> null else staticinvoke(class
>>>>> org.apache.spark.sql.catalyst.util.ArrayBasedMapData$, ObjectType(interface
>>>>> scala.collection.Map), toScalaMap, staticinvoke(class
>>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>>> scala.collection.Seq), make,
>>>>> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>>>> StringType), lambdavariable(MapObjects_loopValue16,
>>>>> MapObjects_loopIsNull17, StringType).toString, input[0,
>>>>> map<string,string>].keyArray).array, true), staticinvoke(class
>>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>>> scala.collection.Seq), make,
>>>>> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>>>> StringType), lambdavariable(MapObjects_loopValue18,
>>>>> MapObjects_loopIsNull19, StringType).toString, input[0,
>>>>> map<string,string>].valueArray).array, true), true), if (isnull(input[1,
>>>>> array<string>])) null else staticinvoke(class
>>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>>> scala.collection.Seq), make,
>>>>> mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
>>>>> StringType), lambdavariable(MapObjects_loopValue20,
>>>>> MapObjects_loopIsNull21, StringType).toString, input[1,
>>>>> array<string>]).array, true),
>>>>> StructField(_1,MapType(StringType,StringType,true),true),
>>>>> StructField(_2,ArrayType(StringType,true),true))
>>>>> [info] :- if (isnull(input[0, map<string,string>])) null else
>>>>> staticinvoke(class org.apache.spark.sql.catalyst.util.ArrayBasedMapData$,
>>>>> ObjectType(interface scala.collection.Map), toScalaMap, staticinvoke(class
>>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>>> scala.collection.Seq), make,
>>>>> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>>>> StringType), lambdavariable(MapObjects_loopValue16,
>>>>> MapObjects_loopIsNull17, StringType).toString, input[0,
>>>>> map<string,string>].keyArray).array, true), staticinvoke(class
>>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>>> scala.collection.Seq), make,
>>>>> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>>>> StringType), lambdavariable(MapObjects_loopValue18,
>>>>> MapObjects_loopIsNull19, StringType).toString, input[0,
>>>>> map<string,string>].valueArray).array, true), true)
>>>>> [info] :  :- isnull(input[0, map<string,string>])
>>>>> [info] :  :  +- input[0, map<string,string>]
>>>>> [info] :  :- null
>>>>> [info] :  +- staticinvoke(class
>>>>> org.apache.spark.sql.catalyst.util.ArrayBasedMapData$, ObjectType(interface
>>>>> scala.collection.Map), toScalaMap, staticinvoke(class
>>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>>> scala.collection.Seq), make,
>>>>> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>>>> StringType), lambdavariable(MapObjects_loopValue16,
>>>>> MapObjects_loopIsNull17, StringType).toString, input[0,
>>>>> map<string,string>].keyArray).array, true), staticinvoke(class
>>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>>> scala.collection.Seq), make,
>>>>> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>>>> StringType), lambdavariable(MapObjects_loopValue18,
>>>>> MapObjects_loopIsNull19, StringType).toString, input[0,
>>>>> map<string,string>].valueArray).array, true), true)
>>>>> [info] :     :- staticinvoke(class
>>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>>> scala.collection.Seq), make,
>>>>> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>>>> StringType), lambdavariable(MapObjects_loopValue16,
>>>>> MapObjects_loopIsNull17, StringType).toString, input[0,
>>>>> map<string,string>].keyArray).array, true)
>>>>> [info] :     :  +- mapobjects(lambdavariable(MapObjects_loopValue16,
>>>>> MapObjects_loopIsNull17, StringType),
>>>>> lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>>>> StringType).toString, input[0, map<string,string>].keyArray).array
>>>>> [info] :     :     +-
>>>>> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>>>> StringType), lambdavariable(MapObjects_loopValue16,
>>>>> MapObjects_loopIsNull17, StringType).toString, input[0,
>>>>> map<string,string>].keyArray)
>>>>> [info] :     :        :- lambdavariable(MapObjects_loopValue16,
>>>>> MapObjects_loopIsNull17, StringType).toString
>>>>> [info] :     :        :  +- lambdavariable(MapObjects_loopValue16,
>>>>> MapObjects_loopIsNull17, StringType)
>>>>> [info] :     :        +- input[0, map<string,string>].keyArray
>>>>> [info] :     :           +- input[0, map<string,string>]
>>>>> [info] :     +- staticinvoke(class
>>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>>> scala.collection.Seq), make,
>>>>> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>>>> StringType), lambdavariable(MapObjects_loopValue18,
>>>>> MapObjects_loopIsNull19, StringType).toString, input[0,
>>>>> map<string,string>].valueArray).array, true)
>>>>> [info] :        +- mapobjects(lambdavariable(MapObjects_loopValue18,
>>>>> MapObjects_loopIsNull19, StringType),
>>>>> lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>>>> StringType).toString, input[0, map<string,string>].valueArray).array
>>>>> [info] :           +-
>>>>> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>>>> StringType), lambdavariable(MapObjects_loopValue18,
>>>>> MapObjects_loopIsNull19, StringType).toString, input[0,
>>>>> map<string,string>].valueArray)
>>>>> [info] :              :- lambdavariable(MapObjects_loopValue18,
>>>>> MapObjects_loopIsNull19, StringType).toString
>>>>> [info] :              :  +- lambdavariable(MapObjects_loopValue18,
>>>>> MapObjects_loopIsNull19, StringType)
>>>>> [info] :              +- input[0, map<string,string>].valueArray
>>>>> [info] :                 +- input[0, map<string,string>]
>>>>> [info] +- if (isnull(input[1, array<string>])) null else
>>>>> staticinvoke(class scala.collection.mutable.WrappedArray$,
>>>>> ObjectType(interface scala.collection.Seq), make,
>>>>> mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
>>>>> StringType), lambdavariable(MapObjects_loopValue20,
>>>>> MapObjects_loopIsNull21, StringType).toString, input[1,
>>>>> array<string>]).array, true)
>>>>> [info]    :- isnull(input[1, array<string>])
>>>>> [info]    :  +- input[1, array<string>]
>>>>> [info]    :- null
>>>>> [info]    +- staticinvoke(class
>>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>>> scala.collection.Seq), make,
>>>>> mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
>>>>> StringType), lambdavariable(MapObjects_loopValue20,
>>>>> MapObjects_loopIsNull21, StringType).toString, input[1,
>>>>> array<string>]).array, true)
>>>>> [info]       +- mapobjects(lambdavariable(MapObjects_loopValue20,
>>>>> MapObjects_loopIsNull21, StringType),
>>>>> lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
>>>>> StringType).toString, input[1, array<string>]).array
>>>>> [info]          +- mapobjects(lambdavariable(MapObjects_loopValue20,
>>>>> MapObjects_loopIsNull21, StringType),
>>>>> lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
>>>>> StringType).toString, input[1, array<string>])
>>>>> [info]             :- lambdavariable(MapObjects_loopValue20,
>>>>> MapObjects_loopIsNull21, StringType).toString
>>>>> [info]             :  +- lambdavariable(MapObjects_loopValue20,
>>>>> MapObjects_loopIsNull21, StringType)
>>>>> [info]             +- input[1, array<string>]
>>>>> [info]   at
>>>>> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:244)
>>>>> [info]   at
>>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
>>>>> [info]   at
>>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
>>>>> [info]   at
>>>>> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
>>>>> [info]   at
>>>>> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
>>>>> [info]   at
>>>>> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
>>>>> [info]   at
>>>>> scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
>>>>> [info]   at
>>>>> scala.collection.TraversableLike$class.map(TraversableLike.scala:245)
>>>>> [info]   at
>>>>> scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186)
>>>>> [info]   at
>>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2121)
>>>>> [info]   at
>>>>> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:54)
>>>>> [info]   at
>>>>> org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2408)
>>>>> [info]   at org.apache.spark.sql.Dataset.org
>>>>> $apache$spark$sql$Dataset$$execute$1(Dataset.scala:2120)
>>>>> [info]   at
>>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Dataset.scala:2125)
>>>>> [info]   at
>>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Dataset.scala:2125)
>>>>> [info]   at
>>>>> org.apache.spark.sql.Dataset.withCallback(Dataset.scala:2421)
>>>>> [info]   at org.apache.spark.sql.Dataset.org
>>>>> $apache$spark$sql$Dataset$$collect(Dataset.scala:2125)
>>>>> [info]   at org.apache.spark.sql.Dataset.collect(Dataset.scala:2101)
>>>>>
>>>>>
>>>>>
>>>>> On Thu, Apr 28, 2016 at 2:41 PM, Ted Yu <yu...@gmail.com> wrote:
>>>>>
>>>>>> Are you able to pastebin a unit test which can reproduce the
>>>>>> following ?
>>>>>>
>>>>>> Thanks
>>>>>>
>>>>>> On Apr 28, 2016, at 11:35 AM, Koert Kuipers <ko...@tresata.com>
>>>>>> wrote:
>>>>>>
>>>>>> i tried for the first time to run our own in-house unit tests on
>>>>>> spark 2, and i get the error below.
>>>>>> has anyone seen this?
>>>>>>
>>>>>> it is reproducible. i tried latest java 7 and it is still there.
>>>>>>
>>>>>> # A fatal error has been detected by the Java Runtime Environment:
>>>>>> #
>>>>>> #  SIGSEGV (0xb) at pc=0x00007f7c3a4b1f54, pid=21939,
>>>>>> tid=140171011417856
>>>>>> #
>>>>>> # JRE version: Java(TM) SE Runtime Environment (7.0_75-b13) (build
>>>>>> 1.7.0_75-b13)
>>>>>> # Java VM: Java HotSpot(TM) 64-Bit Server VM (24.75-b04 mixed mode
>>>>>> linux-amd64 compressed oops)
>>>>>> # Problematic frame:
>>>>>> # V  [libjvm.so+0x747f54]  _Copy_arrayof_conjoint_jlongs+0x44
>>>>>>
>>>>>> more info:
>>>>>>
>>>>>> Stack: [0x00007f7c1b47e000,0x00007f7c1b57f000],
>>>>>> sp=0x00007f7c1b57a9a8,  free space=1010k
>>>>>> Native frames: (J=compiled Java code, j=interpreted, Vv=VM code,
>>>>>> C=native code)
>>>>>> V  [libjvm.so+0x747f54]  _Copy_arrayof_conjoint_jlongs+0x44
>>>>>> j
>>>>>> sun.misc.Unsafe.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V+0
>>>>>> j
>>>>>> org.apache.spark.unsafe.Platform.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V+34
>>>>>> j  org.apache.spark.unsafe.types.UTF8String.getBytes()[B+76
>>>>>> j
>>>>>> org.apache.spark.unsafe.types.UTF8String.toString()Ljava/lang/String;+5
>>>>>> j
>>>>>> org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificSafeProjection.apply(Ljava/lang/Object;)Ljava/lang/Object;+876
>>>>>> j
>>>>>> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(Lorg/apache/spark/sql/catalyst/InternalRow;)Ljava/lang/Object;+5
>>>>>> j
>>>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Lorg/apache/spark/sql/catalyst/InternalRow;)Ljava/lang/Object;+11
>>>>>> j
>>>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Ljava/lang/Object;)Ljava/lang/Object;+5
>>>>>> J 13277 C2
>>>>>> scala.collection.mutable.ArrayOps$ofRef.map(Lscala/Function1;Lscala/collection/generic/CanBuildFrom;)Ljava/lang/Object;
>>>>>> (7 bytes) @ 0x00007f7c25eeae08 [0x00007f7c25eead40+0xc8]
>>>>>> j
>>>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply()Ljava/lang/Object;+43
>>>>>> j
>>>>>> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(Lorg/apache/spark/sql/SparkSession;Lorg/apache/spark/sql/execution/QueryExecution;Lscala/Function0;)Ljava/lang/Object;+106
>>>>>> j
>>>>>> org.apache.spark.sql.Dataset.withNewExecutionId(Lscala/Function0;)Ljava/lang/Object;+12
>>>>>> j  org.apache.spark.sql.Dataset.org
>>>>>> $apache$spark$sql$Dataset$$execute$1()Ljava/lang/Object;+9
>>>>>> j
>>>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Lorg/apache/spark/sql/Dataset;)Ljava/lang/Object;+4
>>>>>> j
>>>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Ljava/lang/Object;)Ljava/lang/Object;+5
>>>>>> j
>>>>>> org.apache.spark.sql.Dataset.withCallback(Ljava/lang/String;Lorg/apache/spark/sql/Dataset;Lscala/Function1;)Ljava/lang/Object;+25
>>>>>> j  org.apache.spark.sql.Dataset.org
>>>>>> $apache$spark$sql$Dataset$$collect(Z)Ljava/lang/Object;+20
>>>>>> j  org.apache.spark.sql.Dataset.collect()Ljava/lang/Object;+2
>>>>>>
>>>>>>
>>>>>
>>>>
>>
>

Re: spark 2 segfault

Posted by Reynold Xin <rx...@databricks.com>.
Definitely looks like a bug.

Ted - are you looking at this?


On Mon, May 2, 2016 at 7:15 AM, Koert Kuipers <ko...@tresata.com> wrote:

> Created issue:
> https://issues.apache.org/jira/browse/SPARK-15062
>
> On Mon, May 2, 2016 at 6:48 AM, Ted Yu <yu...@gmail.com> wrote:
>
>> I tried the same statement using Spark 1.6.1
>> There was no error with default memory setting.
>>
>> Suggest logging a bug.
>>
>> On May 1, 2016, at 9:22 PM, Koert Kuipers <ko...@tresata.com> wrote:
>>
>> Yeah I got that too, then I increased heap for tests to 8G to get error I
>> showed earlier.
>> On May 2, 2016 12:09 AM, "Ted Yu" <yu...@gmail.com> wrote:
>>
>>> Using commit hash 90787de864b58a1079c23e6581381ca8ffe7685f and
>>> Java 1.7.0_67 , I got:
>>>
>>> scala> val dfComplicated = sc.parallelize(List((Map("1" -> "a"),
>>> List("b", "c")), (Map("2" -> "b"), List("d", "e")))).toDF
>>> ...
>>> dfComplicated: org.apache.spark.sql.DataFrame = [_1: map<string,string>,
>>> _2: array<string>]
>>>
>>> scala> dfComplicated.show
>>> java.lang.OutOfMemoryError: Java heap space
>>>   at
>>> org.apache.spark.unsafe.types.UTF8String.getBytes(UTF8String.java:229)
>>>   at
>>> org.apache.spark.unsafe.types.UTF8String.toString(UTF8String.java:821)
>>>   at
>>> org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificSafeProjection.apply(Unknown
>>> Source)
>>>   at
>>> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:241)
>>>   at
>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
>>>   at
>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
>>>   at
>>> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>>>   at
>>> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>>>   at
>>> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
>>>   at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
>>>   at
>>> scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
>>>   at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186)
>>>   at
>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2121)
>>>   at
>>> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:54)
>>>   at org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2408)
>>>   at org.apache.spark.sql.Dataset.org
>>> $apache$spark$sql$Dataset$$execute$1(Dataset.scala:2120)
>>>   at org.apache.spark.sql.Dataset.org
>>> $apache$spark$sql$Dataset$$collect(Dataset.scala:2127)
>>>   at
>>> org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:1861)
>>>   at
>>> org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:1860)
>>>   at org.apache.spark.sql.Dataset.withTypedCallback(Dataset.scala:2438)
>>>   at org.apache.spark.sql.Dataset.head(Dataset.scala:1860)
>>>   at org.apache.spark.sql.Dataset.take(Dataset.scala:2077)
>>>   at org.apache.spark.sql.Dataset.showString(Dataset.scala:238)
>>>   at org.apache.spark.sql.Dataset.show(Dataset.scala:529)
>>>   at org.apache.spark.sql.Dataset.show(Dataset.scala:489)
>>>   at org.apache.spark.sql.Dataset.show(Dataset.scala:498)
>>>   ... 6 elided
>>>
>>> scala>
>>>
>>> On Sun, May 1, 2016 at 8:34 PM, Koert Kuipers <ko...@tresata.com> wrote:
>>>
>>>> by removing dependencies it turns into a different error, see below.
>>>> the test is simply writing a DataFrame out to file and reading it back
>>>> in. i see the error for all data sources (json, parquet, etc.).
>>>>
>>>> this is the data that i write out and read back in:
>>>> val dfComplicated = sc.parallelize(List((Map("1" -> "a"), List("b",
>>>> "c")), (Map("2" -> "b"), List("d", "e")))).toDF
>>>>
>>>>
>>>> [info]   java.lang.RuntimeException: Error while decoding:
>>>> java.lang.NegativeArraySizeException
>>>> [info] createexternalrow(if (isnull(input[0, map<string,string>])) null
>>>> else staticinvoke(class
>>>> org.apache.spark.sql.catalyst.util.ArrayBasedMapData$, ObjectType(interface
>>>> scala.collection.Map), toScalaMap, staticinvoke(class
>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>> scala.collection.Seq), make,
>>>> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>>> StringType), lambdavariable(MapObjects_loopValue16,
>>>> MapObjects_loopIsNull17, StringType).toString, input[0,
>>>> map<string,string>].keyArray).array, true), staticinvoke(class
>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>> scala.collection.Seq), make,
>>>> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>>> StringType), lambdavariable(MapObjects_loopValue18,
>>>> MapObjects_loopIsNull19, StringType).toString, input[0,
>>>> map<string,string>].valueArray).array, true), true), if (isnull(input[1,
>>>> array<string>])) null else staticinvoke(class
>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>> scala.collection.Seq), make,
>>>> mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
>>>> StringType), lambdavariable(MapObjects_loopValue20,
>>>> MapObjects_loopIsNull21, StringType).toString, input[1,
>>>> array<string>]).array, true),
>>>> StructField(_1,MapType(StringType,StringType,true),true),
>>>> StructField(_2,ArrayType(StringType,true),true))
>>>> [info] :- if (isnull(input[0, map<string,string>])) null else
>>>> staticinvoke(class org.apache.spark.sql.catalyst.util.ArrayBasedMapData$,
>>>> ObjectType(interface scala.collection.Map), toScalaMap, staticinvoke(class
>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>> scala.collection.Seq), make,
>>>> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>>> StringType), lambdavariable(MapObjects_loopValue16,
>>>> MapObjects_loopIsNull17, StringType).toString, input[0,
>>>> map<string,string>].keyArray).array, true), staticinvoke(class
>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>> scala.collection.Seq), make,
>>>> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>>> StringType), lambdavariable(MapObjects_loopValue18,
>>>> MapObjects_loopIsNull19, StringType).toString, input[0,
>>>> map<string,string>].valueArray).array, true), true)
>>>> [info] :  :- isnull(input[0, map<string,string>])
>>>> [info] :  :  +- input[0, map<string,string>]
>>>> [info] :  :- null
>>>> [info] :  +- staticinvoke(class
>>>> org.apache.spark.sql.catalyst.util.ArrayBasedMapData$, ObjectType(interface
>>>> scala.collection.Map), toScalaMap, staticinvoke(class
>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>> scala.collection.Seq), make,
>>>> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>>> StringType), lambdavariable(MapObjects_loopValue16,
>>>> MapObjects_loopIsNull17, StringType).toString, input[0,
>>>> map<string,string>].keyArray).array, true), staticinvoke(class
>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>> scala.collection.Seq), make,
>>>> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>>> StringType), lambdavariable(MapObjects_loopValue18,
>>>> MapObjects_loopIsNull19, StringType).toString, input[0,
>>>> map<string,string>].valueArray).array, true), true)
>>>> [info] :     :- staticinvoke(class
>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>> scala.collection.Seq), make,
>>>> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>>> StringType), lambdavariable(MapObjects_loopValue16,
>>>> MapObjects_loopIsNull17, StringType).toString, input[0,
>>>> map<string,string>].keyArray).array, true)
>>>> [info] :     :  +- mapobjects(lambdavariable(MapObjects_loopValue16,
>>>> MapObjects_loopIsNull17, StringType),
>>>> lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>>> StringType).toString, input[0, map<string,string>].keyArray).array
>>>> [info] :     :     +- mapobjects(lambdavariable(MapObjects_loopValue16,
>>>> MapObjects_loopIsNull17, StringType),
>>>> lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>>> StringType).toString, input[0, map<string,string>].keyArray)
>>>> [info] :     :        :- lambdavariable(MapObjects_loopValue16,
>>>> MapObjects_loopIsNull17, StringType).toString
>>>> [info] :     :        :  +- lambdavariable(MapObjects_loopValue16,
>>>> MapObjects_loopIsNull17, StringType)
>>>> [info] :     :        +- input[0, map<string,string>].keyArray
>>>> [info] :     :           +- input[0, map<string,string>]
>>>> [info] :     +- staticinvoke(class
>>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>>> scala.collection.Seq), make,
>>>> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>>> StringType), lambdavariable(MapObjects_loopValue18,
>>>> MapObjects_loopIsNull19, StringType).toString, input[0,
>>>> map<string,string>].valueArray).array, true)
>>>> [info] :        +- mapobjects(lambdavariable(MapObjects_loopValue18,
>>>> MapObjects_loopIsNull19, StringType),
>>>> lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>>> StringType).toString, input[0, map<string,string>].valueArray).array
>>>> [info] :           +- mapobjects(lambdavariable(MapObjects_loopValue18,
>>>> MapObjects_loopIsNull19, StringType),
>>>> lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>>> StringType).toString, input[0, map<string,string>].valueArray)
>>>> [info] :              :- lambdavariable(MapObjects_loopValue18,
>>>> MapObjects_loopIsNull19, StringType).toString
>>>> [info] :              :  +- lambdavariable(MapObjects_loopValue18,
>>>> MapObjects_loopIsNull19, StringType)
>>>> [info] :              +- input[0, map<string,string>].valueArray
>>>> [info] :                 +- input[0, map<string,string>]
>>>> [info] +- if (isnull(input[1, array<string>])) null else
>>>> staticinvoke(class scala.collection.mutable.WrappedArray$,
>>>> ObjectType(interface scala.collection.Seq), make,
>>>> mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
>>>> StringType), lambdavariable(MapObjects_loopValue20,
>>>> MapObjects_loopIsNull21, StringType).toString, input[1,
>>>> array<string>]).array, true)
>>>> [info]    :- isnull(input[1, array<string>])
>>>> [info]    :  +- input[1, array<string>]
>>>> [info]    :- null
>>>> [info]    +- staticinvoke(class scala.collection.mutable.WrappedArray$,
>>>> ObjectType(interface scala.collection.Seq), make,
>>>> mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
>>>> StringType), lambdavariable(MapObjects_loopValue20,
>>>> MapObjects_loopIsNull21, StringType).toString, input[1,
>>>> array<string>]).array, true)
>>>> [info]       +- mapobjects(lambdavariable(MapObjects_loopValue20,
>>>> MapObjects_loopIsNull21, StringType),
>>>> lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
>>>> StringType).toString, input[1, array<string>]).array
>>>> [info]          +- mapobjects(lambdavariable(MapObjects_loopValue20,
>>>> MapObjects_loopIsNull21, StringType),
>>>> lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
>>>> StringType).toString, input[1, array<string>])
>>>> [info]             :- lambdavariable(MapObjects_loopValue20,
>>>> MapObjects_loopIsNull21, StringType).toString
>>>> [info]             :  +- lambdavariable(MapObjects_loopValue20,
>>>> MapObjects_loopIsNull21, StringType)
>>>> [info]             +- input[1, array<string>]
>>>> [info]   at
>>>> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:244)
>>>> [info]   at
>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
>>>> [info]   at
>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
>>>> [info]   at
>>>> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
>>>> [info]   at
>>>> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
>>>> [info]   at
>>>> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
>>>> [info]   at
>>>> scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
>>>> [info]   at
>>>> scala.collection.TraversableLike$class.map(TraversableLike.scala:245)
>>>> [info]   at
>>>> scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186)
>>>> [info]   at
>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2121)
>>>> [info]   at
>>>> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:54)
>>>> [info]   at
>>>> org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2408)
>>>> [info]   at org.apache.spark.sql.Dataset.org
>>>> $apache$spark$sql$Dataset$$execute$1(Dataset.scala:2120)
>>>> [info]   at
>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Dataset.scala:2125)
>>>> [info]   at
>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Dataset.scala:2125)
>>>> [info]   at
>>>> org.apache.spark.sql.Dataset.withCallback(Dataset.scala:2421)
>>>> [info]   at org.apache.spark.sql.Dataset.org
>>>> $apache$spark$sql$Dataset$$collect(Dataset.scala:2125)
>>>> [info]   at org.apache.spark.sql.Dataset.collect(Dataset.scala:2101)
>>>>
>>>>
>>>>
>>>> On Thu, Apr 28, 2016 at 2:41 PM, Ted Yu <yu...@gmail.com> wrote:
>>>>
>>>>> Are you able to pastebin a unit test which can reproduce the following
>>>>> ?
>>>>>
>>>>> Thanks
>>>>>
>>>>> On Apr 28, 2016, at 11:35 AM, Koert Kuipers <ko...@tresata.com> wrote:
>>>>>
>>>>> i tried for the first time to run our own in-house unit tests on spark
>>>>> 2, and i get the error below.
>>>>> has anyone seen this?
>>>>>
>>>>> it is reproducible. i tried latest java 7 and it is still there.
>>>>>
>>>>> # A fatal error has been detected by the Java Runtime Environment:
>>>>> #
>>>>> #  SIGSEGV (0xb) at pc=0x00007f7c3a4b1f54, pid=21939,
>>>>> tid=140171011417856
>>>>> #
>>>>> # JRE version: Java(TM) SE Runtime Environment (7.0_75-b13) (build
>>>>> 1.7.0_75-b13)
>>>>> # Java VM: Java HotSpot(TM) 64-Bit Server VM (24.75-b04 mixed mode
>>>>> linux-amd64 compressed oops)
>>>>> # Problematic frame:
>>>>> # V  [libjvm.so+0x747f54]  _Copy_arrayof_conjoint_jlongs+0x44
>>>>>
>>>>> more info:
>>>>>
>>>>> Stack: [0x00007f7c1b47e000,0x00007f7c1b57f000],
>>>>> sp=0x00007f7c1b57a9a8,  free space=1010k
>>>>> Native frames: (J=compiled Java code, j=interpreted, Vv=VM code,
>>>>> C=native code)
>>>>> V  [libjvm.so+0x747f54]  _Copy_arrayof_conjoint_jlongs+0x44
>>>>> j
>>>>> sun.misc.Unsafe.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V+0
>>>>> j
>>>>> org.apache.spark.unsafe.Platform.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V+34
>>>>> j  org.apache.spark.unsafe.types.UTF8String.getBytes()[B+76
>>>>> j
>>>>> org.apache.spark.unsafe.types.UTF8String.toString()Ljava/lang/String;+5
>>>>> j
>>>>> org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificSafeProjection.apply(Ljava/lang/Object;)Ljava/lang/Object;+876
>>>>> j
>>>>> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(Lorg/apache/spark/sql/catalyst/InternalRow;)Ljava/lang/Object;+5
>>>>> j
>>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Lorg/apache/spark/sql/catalyst/InternalRow;)Ljava/lang/Object;+11
>>>>> j
>>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Ljava/lang/Object;)Ljava/lang/Object;+5
>>>>> J 13277 C2
>>>>> scala.collection.mutable.ArrayOps$ofRef.map(Lscala/Function1;Lscala/collection/generic/CanBuildFrom;)Ljava/lang/Object;
>>>>> (7 bytes) @ 0x00007f7c25eeae08 [0x00007f7c25eead40+0xc8]
>>>>> j
>>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply()Ljava/lang/Object;+43
>>>>> j
>>>>> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(Lorg/apache/spark/sql/SparkSession;Lorg/apache/spark/sql/execution/QueryExecution;Lscala/Function0;)Ljava/lang/Object;+106
>>>>> j
>>>>> org.apache.spark.sql.Dataset.withNewExecutionId(Lscala/Function0;)Ljava/lang/Object;+12
>>>>> j  org.apache.spark.sql.Dataset.org
>>>>> $apache$spark$sql$Dataset$$execute$1()Ljava/lang/Object;+9
>>>>> j
>>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Lorg/apache/spark/sql/Dataset;)Ljava/lang/Object;+4
>>>>> j
>>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Ljava/lang/Object;)Ljava/lang/Object;+5
>>>>> j
>>>>> org.apache.spark.sql.Dataset.withCallback(Ljava/lang/String;Lorg/apache/spark/sql/Dataset;Lscala/Function1;)Ljava/lang/Object;+25
>>>>> j  org.apache.spark.sql.Dataset.org
>>>>> $apache$spark$sql$Dataset$$collect(Z)Ljava/lang/Object;+20
>>>>> j  org.apache.spark.sql.Dataset.collect()Ljava/lang/Object;+2
>>>>>
>>>>>
>>>>
>>>
>

Re: spark 2 segfault

Posted by Koert Kuipers <ko...@tresata.com>.
Created issue:
https://issues.apache.org/jira/browse/SPARK-15062

On Mon, May 2, 2016 at 6:48 AM, Ted Yu <yu...@gmail.com> wrote:

> I tried the same statement using Spark 1.6.1
> There was no error with default memory setting.
>
> Suggest logging a bug.
>
> On May 1, 2016, at 9:22 PM, Koert Kuipers <ko...@tresata.com> wrote:
>
> Yeah I got that too, then I increased heap for tests to 8G to get error I
> showed earlier.
> On May 2, 2016 12:09 AM, "Ted Yu" <yu...@gmail.com> wrote:
>
>> Using commit hash 90787de864b58a1079c23e6581381ca8ffe7685f and
>> Java 1.7.0_67 , I got:
>>
>> scala> val dfComplicated = sc.parallelize(List((Map("1" -> "a"),
>> List("b", "c")), (Map("2" -> "b"), List("d", "e")))).toDF
>> ...
>> dfComplicated: org.apache.spark.sql.DataFrame = [_1: map<string,string>,
>> _2: array<string>]
>>
>> scala> dfComplicated.show
>> java.lang.OutOfMemoryError: Java heap space
>>   at
>> org.apache.spark.unsafe.types.UTF8String.getBytes(UTF8String.java:229)
>>   at
>> org.apache.spark.unsafe.types.UTF8String.toString(UTF8String.java:821)
>>   at
>> org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificSafeProjection.apply(Unknown
>> Source)
>>   at
>> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:241)
>>   at
>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
>>   at
>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
>>   at
>> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>>   at
>> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>>   at
>> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
>>   at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
>>   at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
>>   at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186)
>>   at
>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2121)
>>   at
>> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:54)
>>   at org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2408)
>>   at org.apache.spark.sql.Dataset.org
>> $apache$spark$sql$Dataset$$execute$1(Dataset.scala:2120)
>>   at org.apache.spark.sql.Dataset.org
>> $apache$spark$sql$Dataset$$collect(Dataset.scala:2127)
>>   at
>> org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:1861)
>>   at
>> org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:1860)
>>   at org.apache.spark.sql.Dataset.withTypedCallback(Dataset.scala:2438)
>>   at org.apache.spark.sql.Dataset.head(Dataset.scala:1860)
>>   at org.apache.spark.sql.Dataset.take(Dataset.scala:2077)
>>   at org.apache.spark.sql.Dataset.showString(Dataset.scala:238)
>>   at org.apache.spark.sql.Dataset.show(Dataset.scala:529)
>>   at org.apache.spark.sql.Dataset.show(Dataset.scala:489)
>>   at org.apache.spark.sql.Dataset.show(Dataset.scala:498)
>>   ... 6 elided
>>
>> scala>
>>
>> On Sun, May 1, 2016 at 8:34 PM, Koert Kuipers <ko...@tresata.com> wrote:
>>
>>> by removing dependencies it turns into a different error, see below.
>>> the test is simply writing a DataFrame out to file and reading it back
>>> in. i see the error for all data sources (json, parquet, etc.).
>>>
>>> this is the data that i write out and read back in:
>>> val dfComplicated = sc.parallelize(List((Map("1" -> "a"), List("b",
>>> "c")), (Map("2" -> "b"), List("d", "e")))).toDF
>>>
>>>
>>> [info]   java.lang.RuntimeException: Error while decoding:
>>> java.lang.NegativeArraySizeException
>>> [info] createexternalrow(if (isnull(input[0, map<string,string>])) null
>>> else staticinvoke(class
>>> org.apache.spark.sql.catalyst.util.ArrayBasedMapData$, ObjectType(interface
>>> scala.collection.Map), toScalaMap, staticinvoke(class
>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>> scala.collection.Seq), make,
>>> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>> StringType), lambdavariable(MapObjects_loopValue16,
>>> MapObjects_loopIsNull17, StringType).toString, input[0,
>>> map<string,string>].keyArray).array, true), staticinvoke(class
>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>> scala.collection.Seq), make,
>>> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>> StringType), lambdavariable(MapObjects_loopValue18,
>>> MapObjects_loopIsNull19, StringType).toString, input[0,
>>> map<string,string>].valueArray).array, true), true), if (isnull(input[1,
>>> array<string>])) null else staticinvoke(class
>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>> scala.collection.Seq), make,
>>> mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
>>> StringType), lambdavariable(MapObjects_loopValue20,
>>> MapObjects_loopIsNull21, StringType).toString, input[1,
>>> array<string>]).array, true),
>>> StructField(_1,MapType(StringType,StringType,true),true),
>>> StructField(_2,ArrayType(StringType,true),true))
>>> [info] :- if (isnull(input[0, map<string,string>])) null else
>>> staticinvoke(class org.apache.spark.sql.catalyst.util.ArrayBasedMapData$,
>>> ObjectType(interface scala.collection.Map), toScalaMap, staticinvoke(class
>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>> scala.collection.Seq), make,
>>> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>> StringType), lambdavariable(MapObjects_loopValue16,
>>> MapObjects_loopIsNull17, StringType).toString, input[0,
>>> map<string,string>].keyArray).array, true), staticinvoke(class
>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>> scala.collection.Seq), make,
>>> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>> StringType), lambdavariable(MapObjects_loopValue18,
>>> MapObjects_loopIsNull19, StringType).toString, input[0,
>>> map<string,string>].valueArray).array, true), true)
>>> [info] :  :- isnull(input[0, map<string,string>])
>>> [info] :  :  +- input[0, map<string,string>]
>>> [info] :  :- null
>>> [info] :  +- staticinvoke(class
>>> org.apache.spark.sql.catalyst.util.ArrayBasedMapData$, ObjectType(interface
>>> scala.collection.Map), toScalaMap, staticinvoke(class
>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>> scala.collection.Seq), make,
>>> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>> StringType), lambdavariable(MapObjects_loopValue16,
>>> MapObjects_loopIsNull17, StringType).toString, input[0,
>>> map<string,string>].keyArray).array, true), staticinvoke(class
>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>> scala.collection.Seq), make,
>>> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>> StringType), lambdavariable(MapObjects_loopValue18,
>>> MapObjects_loopIsNull19, StringType).toString, input[0,
>>> map<string,string>].valueArray).array, true), true)
>>> [info] :     :- staticinvoke(class
>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>> scala.collection.Seq), make,
>>> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>> StringType), lambdavariable(MapObjects_loopValue16,
>>> MapObjects_loopIsNull17, StringType).toString, input[0,
>>> map<string,string>].keyArray).array, true)
>>> [info] :     :  +- mapobjects(lambdavariable(MapObjects_loopValue16,
>>> MapObjects_loopIsNull17, StringType),
>>> lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>> StringType).toString, input[0, map<string,string>].keyArray).array
>>> [info] :     :     +- mapobjects(lambdavariable(MapObjects_loopValue16,
>>> MapObjects_loopIsNull17, StringType),
>>> lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
>>> StringType).toString, input[0, map<string,string>].keyArray)
>>> [info] :     :        :- lambdavariable(MapObjects_loopValue16,
>>> MapObjects_loopIsNull17, StringType).toString
>>> [info] :     :        :  +- lambdavariable(MapObjects_loopValue16,
>>> MapObjects_loopIsNull17, StringType)
>>> [info] :     :        +- input[0, map<string,string>].keyArray
>>> [info] :     :           +- input[0, map<string,string>]
>>> [info] :     +- staticinvoke(class
>>> scala.collection.mutable.WrappedArray$, ObjectType(interface
>>> scala.collection.Seq), make,
>>> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>> StringType), lambdavariable(MapObjects_loopValue18,
>>> MapObjects_loopIsNull19, StringType).toString, input[0,
>>> map<string,string>].valueArray).array, true)
>>> [info] :        +- mapobjects(lambdavariable(MapObjects_loopValue18,
>>> MapObjects_loopIsNull19, StringType),
>>> lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>> StringType).toString, input[0, map<string,string>].valueArray).array
>>> [info] :           +- mapobjects(lambdavariable(MapObjects_loopValue18,
>>> MapObjects_loopIsNull19, StringType),
>>> lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
>>> StringType).toString, input[0, map<string,string>].valueArray)
>>> [info] :              :- lambdavariable(MapObjects_loopValue18,
>>> MapObjects_loopIsNull19, StringType).toString
>>> [info] :              :  +- lambdavariable(MapObjects_loopValue18,
>>> MapObjects_loopIsNull19, StringType)
>>> [info] :              +- input[0, map<string,string>].valueArray
>>> [info] :                 +- input[0, map<string,string>]
>>> [info] +- if (isnull(input[1, array<string>])) null else
>>> staticinvoke(class scala.collection.mutable.WrappedArray$,
>>> ObjectType(interface scala.collection.Seq), make,
>>> mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
>>> StringType), lambdavariable(MapObjects_loopValue20,
>>> MapObjects_loopIsNull21, StringType).toString, input[1,
>>> array<string>]).array, true)
>>> [info]    :- isnull(input[1, array<string>])
>>> [info]    :  +- input[1, array<string>]
>>> [info]    :- null
>>> [info]    +- staticinvoke(class scala.collection.mutable.WrappedArray$,
>>> ObjectType(interface scala.collection.Seq), make,
>>> mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
>>> StringType), lambdavariable(MapObjects_loopValue20,
>>> MapObjects_loopIsNull21, StringType).toString, input[1,
>>> array<string>]).array, true)
>>> [info]       +- mapobjects(lambdavariable(MapObjects_loopValue20,
>>> MapObjects_loopIsNull21, StringType),
>>> lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
>>> StringType).toString, input[1, array<string>]).array
>>> [info]          +- mapobjects(lambdavariable(MapObjects_loopValue20,
>>> MapObjects_loopIsNull21, StringType),
>>> lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
>>> StringType).toString, input[1, array<string>])
>>> [info]             :- lambdavariable(MapObjects_loopValue20,
>>> MapObjects_loopIsNull21, StringType).toString
>>> [info]             :  +- lambdavariable(MapObjects_loopValue20,
>>> MapObjects_loopIsNull21, StringType)
>>> [info]             +- input[1, array<string>]
>>> [info]   at
>>> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:244)
>>> [info]   at
>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
>>> [info]   at
>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
>>> [info]   at
>>> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
>>> [info]   at
>>> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
>>> [info]   at
>>> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
>>> [info]   at
>>> scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
>>> [info]   at
>>> scala.collection.TraversableLike$class.map(TraversableLike.scala:245)
>>> [info]   at
>>> scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186)
>>> [info]   at
>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2121)
>>> [info]   at
>>> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:54)
>>> [info]   at
>>> org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2408)
>>> [info]   at org.apache.spark.sql.Dataset.org
>>> $apache$spark$sql$Dataset$$execute$1(Dataset.scala:2120)
>>> [info]   at
>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Dataset.scala:2125)
>>> [info]   at
>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Dataset.scala:2125)
>>> [info]   at org.apache.spark.sql.Dataset.withCallback(Dataset.scala:2421)
>>> [info]   at org.apache.spark.sql.Dataset.org
>>> $apache$spark$sql$Dataset$$collect(Dataset.scala:2125)
>>> [info]   at org.apache.spark.sql.Dataset.collect(Dataset.scala:2101)
>>>
>>>
>>>
>>> On Thu, Apr 28, 2016 at 2:41 PM, Ted Yu <yu...@gmail.com> wrote:
>>>
>>>> Are you able to pastebin a unit test which can reproduce the following ?
>>>>
>>>> Thanks
>>>>
>>>> On Apr 28, 2016, at 11:35 AM, Koert Kuipers <ko...@tresata.com> wrote:
>>>>
>>>> i tried for the first time to run our own in-house unit tests on spark
>>>> 2, and i get the error below.
>>>> has anyone seen this?
>>>>
>>>> it is reproducible. i tried latest java 7 and it is still there.
>>>>
>>>> # A fatal error has been detected by the Java Runtime Environment:
>>>> #
>>>> #  SIGSEGV (0xb) at pc=0x00007f7c3a4b1f54, pid=21939,
>>>> tid=140171011417856
>>>> #
>>>> # JRE version: Java(TM) SE Runtime Environment (7.0_75-b13) (build
>>>> 1.7.0_75-b13)
>>>> # Java VM: Java HotSpot(TM) 64-Bit Server VM (24.75-b04 mixed mode
>>>> linux-amd64 compressed oops)
>>>> # Problematic frame:
>>>> # V  [libjvm.so+0x747f54]  _Copy_arrayof_conjoint_jlongs+0x44
>>>>
>>>> more info:
>>>>
>>>> Stack: [0x00007f7c1b47e000,0x00007f7c1b57f000],
>>>> sp=0x00007f7c1b57a9a8,  free space=1010k
>>>> Native frames: (J=compiled Java code, j=interpreted, Vv=VM code,
>>>> C=native code)
>>>> V  [libjvm.so+0x747f54]  _Copy_arrayof_conjoint_jlongs+0x44
>>>> j
>>>> sun.misc.Unsafe.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V+0
>>>> j
>>>> org.apache.spark.unsafe.Platform.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V+34
>>>> j  org.apache.spark.unsafe.types.UTF8String.getBytes()[B+76
>>>> j
>>>> org.apache.spark.unsafe.types.UTF8String.toString()Ljava/lang/String;+5
>>>> j
>>>> org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificSafeProjection.apply(Ljava/lang/Object;)Ljava/lang/Object;+876
>>>> j
>>>> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(Lorg/apache/spark/sql/catalyst/InternalRow;)Ljava/lang/Object;+5
>>>> j
>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Lorg/apache/spark/sql/catalyst/InternalRow;)Ljava/lang/Object;+11
>>>> j
>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Ljava/lang/Object;)Ljava/lang/Object;+5
>>>> J 13277 C2
>>>> scala.collection.mutable.ArrayOps$ofRef.map(Lscala/Function1;Lscala/collection/generic/CanBuildFrom;)Ljava/lang/Object;
>>>> (7 bytes) @ 0x00007f7c25eeae08 [0x00007f7c25eead40+0xc8]
>>>> j
>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply()Ljava/lang/Object;+43
>>>> j
>>>> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(Lorg/apache/spark/sql/SparkSession;Lorg/apache/spark/sql/execution/QueryExecution;Lscala/Function0;)Ljava/lang/Object;+106
>>>> j
>>>> org.apache.spark.sql.Dataset.withNewExecutionId(Lscala/Function0;)Ljava/lang/Object;+12
>>>> j  org.apache.spark.sql.Dataset.org
>>>> $apache$spark$sql$Dataset$$execute$1()Ljava/lang/Object;+9
>>>> j
>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Lorg/apache/spark/sql/Dataset;)Ljava/lang/Object;+4
>>>> j
>>>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Ljava/lang/Object;)Ljava/lang/Object;+5
>>>> j
>>>> org.apache.spark.sql.Dataset.withCallback(Ljava/lang/String;Lorg/apache/spark/sql/Dataset;Lscala/Function1;)Ljava/lang/Object;+25
>>>> j  org.apache.spark.sql.Dataset.org
>>>> $apache$spark$sql$Dataset$$collect(Z)Ljava/lang/Object;+20
>>>> j  org.apache.spark.sql.Dataset.collect()Ljava/lang/Object;+2
>>>>
>>>>
>>>
>>

Re: spark 2 segfault

Posted by Ted Yu <yu...@gmail.com>.
I tried the same statement using Spark 1.6.1
There was no error with default memory setting. 

Suggest logging a bug. 

> On May 1, 2016, at 9:22 PM, Koert Kuipers <ko...@tresata.com> wrote:
> 
> Yeah I got that too, then I increased heap for tests to 8G to get error I showed earlier.
> 
>> On May 2, 2016 12:09 AM, "Ted Yu" <yu...@gmail.com> wrote:
>> Using commit hash 90787de864b58a1079c23e6581381ca8ffe7685f and Java 1.7.0_67 , I got:
>> 
>> scala> val dfComplicated = sc.parallelize(List((Map("1" -> "a"), List("b", "c")), (Map("2" -> "b"), List("d", "e")))).toDF
>> ...
>> dfComplicated: org.apache.spark.sql.DataFrame = [_1: map<string,string>, _2: array<string>]
>> 
>> scala> dfComplicated.show
>> java.lang.OutOfMemoryError: Java heap space
>>   at org.apache.spark.unsafe.types.UTF8String.getBytes(UTF8String.java:229)
>>   at org.apache.spark.unsafe.types.UTF8String.toString(UTF8String.java:821)
>>   at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificSafeProjection.apply(Unknown Source)
>>   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:241)
>>   at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
>>   at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
>>   at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>>   at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>>   at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
>>   at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
>>   at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
>>   at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186)
>>   at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2121)
>>   at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:54)
>>   at org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2408)
>>   at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$execute$1(Dataset.scala:2120)
>>   at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$collect(Dataset.scala:2127)
>>   at org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:1861)
>>   at org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:1860)
>>   at org.apache.spark.sql.Dataset.withTypedCallback(Dataset.scala:2438)
>>   at org.apache.spark.sql.Dataset.head(Dataset.scala:1860)
>>   at org.apache.spark.sql.Dataset.take(Dataset.scala:2077)
>>   at org.apache.spark.sql.Dataset.showString(Dataset.scala:238)
>>   at org.apache.spark.sql.Dataset.show(Dataset.scala:529)
>>   at org.apache.spark.sql.Dataset.show(Dataset.scala:489)
>>   at org.apache.spark.sql.Dataset.show(Dataset.scala:498)
>>   ... 6 elided
>> 
>> scala>
>> 
>>> On Sun, May 1, 2016 at 8:34 PM, Koert Kuipers <ko...@tresata.com> wrote:
>>> by removing dependencies it turns into a different error, see below.
>>> the test is simply writing a DataFrame out to file and reading it back in. i see the error for all data sources (json, parquet, etc.).
>>> 
>>> this is the data that i write out and read back in:
>>> val dfComplicated = sc.parallelize(List((Map("1" -> "a"), List("b", "c")), (Map("2" -> "b"), List("d", "e")))).toDF 
>>> 
>>> 
>>> [info]   java.lang.RuntimeException: Error while decoding: java.lang.NegativeArraySizeException
>>> [info] createexternalrow(if (isnull(input[0, map<string,string>])) null else staticinvoke(class org.apache.spark.sql.catalyst.util.ArrayBasedMapData$, ObjectType(interface scala.collection.Map), toScalaMap, staticinvoke(class scala.collection.mutable.WrappedArray$, ObjectType(interface scala.collection.Seq), make, mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17, StringType), lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17, StringType).toString, input[0, map<string,string>].keyArray).array, true), staticinvoke(class scala.collection.mutable.WrappedArray$, ObjectType(interface scala.collection.Seq), make, mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19, StringType), lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19, StringType).toString, input[0, map<string,string>].valueArray).array, true), true), if (isnull(input[1, array<string>])) null else staticinvoke(class scala.collection.mutable.WrappedArray$, ObjectType(interface scala.collection.Seq), make, mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21, StringType), lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21, StringType).toString, input[1, array<string>]).array, true), StructField(_1,MapType(StringType,StringType,true),true), StructField(_2,ArrayType(StringType,true),true))
>>> [info] :- if (isnull(input[0, map<string,string>])) null else staticinvoke(class org.apache.spark.sql.catalyst.util.ArrayBasedMapData$, ObjectType(interface scala.collection.Map), toScalaMap, staticinvoke(class scala.collection.mutable.WrappedArray$, ObjectType(interface scala.collection.Seq), make, mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17, StringType), lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17, StringType).toString, input[0, map<string,string>].keyArray).array, true), staticinvoke(class scala.collection.mutable.WrappedArray$, ObjectType(interface scala.collection.Seq), make, mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19, StringType), lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19, StringType).toString, input[0, map<string,string>].valueArray).array, true), true)
>>> [info] :  :- isnull(input[0, map<string,string>])
>>> [info] :  :  +- input[0, map<string,string>]
>>> [info] :  :- null
>>> [info] :  +- staticinvoke(class org.apache.spark.sql.catalyst.util.ArrayBasedMapData$, ObjectType(interface scala.collection.Map), toScalaMap, staticinvoke(class scala.collection.mutable.WrappedArray$, ObjectType(interface scala.collection.Seq), make, mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17, StringType), lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17, StringType).toString, input[0, map<string,string>].keyArray).array, true), staticinvoke(class scala.collection.mutable.WrappedArray$, ObjectType(interface scala.collection.Seq), make, mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19, StringType), lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19, StringType).toString, input[0, map<string,string>].valueArray).array, true), true)
>>> [info] :     :- staticinvoke(class scala.collection.mutable.WrappedArray$, ObjectType(interface scala.collection.Seq), make, mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17, StringType), lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17, StringType).toString, input[0, map<string,string>].keyArray).array, true)
>>> [info] :     :  +- mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17, StringType), lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17, StringType).toString, input[0, map<string,string>].keyArray).array
>>> [info] :     :     +- mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17, StringType), lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17, StringType).toString, input[0, map<string,string>].keyArray)
>>> [info] :     :        :- lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17, StringType).toString
>>> [info] :     :        :  +- lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17, StringType)
>>> [info] :     :        +- input[0, map<string,string>].keyArray
>>> [info] :     :           +- input[0, map<string,string>]
>>> [info] :     +- staticinvoke(class scala.collection.mutable.WrappedArray$, ObjectType(interface scala.collection.Seq), make, mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19, StringType), lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19, StringType).toString, input[0, map<string,string>].valueArray).array, true)
>>> [info] :        +- mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19, StringType), lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19, StringType).toString, input[0, map<string,string>].valueArray).array
>>> [info] :           +- mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19, StringType), lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19, StringType).toString, input[0, map<string,string>].valueArray)
>>> [info] :              :- lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19, StringType).toString
>>> [info] :              :  +- lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19, StringType)
>>> [info] :              +- input[0, map<string,string>].valueArray
>>> [info] :                 +- input[0, map<string,string>]
>>> [info] +- if (isnull(input[1, array<string>])) null else staticinvoke(class scala.collection.mutable.WrappedArray$, ObjectType(interface scala.collection.Seq), make, mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21, StringType), lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21, StringType).toString, input[1, array<string>]).array, true)
>>> [info]    :- isnull(input[1, array<string>])
>>> [info]    :  +- input[1, array<string>]
>>> [info]    :- null
>>> [info]    +- staticinvoke(class scala.collection.mutable.WrappedArray$, ObjectType(interface scala.collection.Seq), make, mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21, StringType), lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21, StringType).toString, input[1, array<string>]).array, true)
>>> [info]       +- mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21, StringType), lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21, StringType).toString, input[1, array<string>]).array
>>> [info]          +- mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21, StringType), lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21, StringType).toString, input[1, array<string>])
>>> [info]             :- lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21, StringType).toString
>>> [info]             :  +- lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21, StringType)
>>> [info]             +- input[1, array<string>]
>>> [info]   at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:244)
>>> [info]   at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
>>> [info]   at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
>>> [info]   at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
>>> [info]   at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
>>> [info]   at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
>>> [info]   at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
>>> [info]   at scala.collection.TraversableLike$class.map(TraversableLike.scala:245)
>>> [info]   at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186)
>>> [info]   at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2121)
>>> [info]   at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:54)
>>> [info]   at org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2408)
>>> [info]   at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$execute$1(Dataset.scala:2120)
>>> [info]   at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Dataset.scala:2125)
>>> [info]   at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Dataset.scala:2125)
>>> [info]   at org.apache.spark.sql.Dataset.withCallback(Dataset.scala:2421)
>>> [info]   at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$collect(Dataset.scala:2125)
>>> [info]   at org.apache.spark.sql.Dataset.collect(Dataset.scala:2101)
>>> 
>>> 
>>> 
>>>> On Thu, Apr 28, 2016 at 2:41 PM, Ted Yu <yu...@gmail.com> wrote:
>>>> Are you able to pastebin a unit test which can reproduce the following ?
>>>> 
>>>> Thanks
>>>> 
>>>>> On Apr 28, 2016, at 11:35 AM, Koert Kuipers <ko...@tresata.com> wrote:
>>>>> 
>>>>> i tried for the first time to run our own in-house unit tests on spark 2, and i get the error below.
>>>>> has anyone seen this?
>>>>> 
>>>>> it is reproducible. i tried latest java 7 and it is still there.
>>>>> 
>>>>> # A fatal error has been detected by the Java Runtime Environment:
>>>>> #
>>>>> #  SIGSEGV (0xb) at pc=0x00007f7c3a4b1f54, pid=21939, tid=140171011417856
>>>>> #
>>>>> # JRE version: Java(TM) SE Runtime Environment (7.0_75-b13) (build 1.7.0_75-b13)
>>>>> # Java VM: Java HotSpot(TM) 64-Bit Server VM (24.75-b04 mixed mode linux-amd64 compressed oops)
>>>>> # Problematic frame:
>>>>> # V  [libjvm.so+0x747f54]  _Copy_arrayof_conjoint_jlongs+0x44
>>>>> 
>>>>> more info:
>>>>> 
>>>>> Stack: [0x00007f7c1b47e000,0x00007f7c1b57f000],  sp=0x00007f7c1b57a9a8,  free space=1010k
>>>>> Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code)
>>>>> V  [libjvm.so+0x747f54]  _Copy_arrayof_conjoint_jlongs+0x44
>>>>> j  sun.misc.Unsafe.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V+0
>>>>> j  org.apache.spark.unsafe.Platform.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V+34
>>>>> j  org.apache.spark.unsafe.types.UTF8String.getBytes()[B+76
>>>>> j  org.apache.spark.unsafe.types.UTF8String.toString()Ljava/lang/String;+5
>>>>> j  org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificSafeProjection.apply(Ljava/lang/Object;)Ljava/lang/Object;+876
>>>>> j  org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(Lorg/apache/spark/sql/catalyst/InternalRow;)Ljava/lang/Object;+5
>>>>> j  org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Lorg/apache/spark/sql/catalyst/InternalRow;)Ljava/lang/Object;+11
>>>>> j  org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Ljava/lang/Object;)Ljava/lang/Object;+5
>>>>> J 13277 C2 scala.collection.mutable.ArrayOps$ofRef.map(Lscala/Function1;Lscala/collection/generic/CanBuildFrom;)Ljava/lang/Object; (7 bytes) @ 0x00007f7c25eeae08 [0x00007f7c25eead40+0xc8]
>>>>> j  org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply()Ljava/lang/Object;+43
>>>>> j  org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(Lorg/apache/spark/sql/SparkSession;Lorg/apache/spark/sql/execution/QueryExecution;Lscala/Function0;)Ljava/lang/Object;+106
>>>>> j  org.apache.spark.sql.Dataset.withNewExecutionId(Lscala/Function0;)Ljava/lang/Object;+12
>>>>> j  org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$execute$1()Ljava/lang/Object;+9
>>>>> j  org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Lorg/apache/spark/sql/Dataset;)Ljava/lang/Object;+4
>>>>> j  org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Ljava/lang/Object;)Ljava/lang/Object;+5
>>>>> j  org.apache.spark.sql.Dataset.withCallback(Ljava/lang/String;Lorg/apache/spark/sql/Dataset;Lscala/Function1;)Ljava/lang/Object;+25
>>>>> j  org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$collect(Z)Ljava/lang/Object;+20
>>>>> j  org.apache.spark.sql.Dataset.collect()Ljava/lang/Object;+2

Re: spark 2 segfault

Posted by Ted Yu <yu...@gmail.com>.
Using commit hash 90787de864b58a1079c23e6581381ca8ffe7685f and
Java 1.7.0_67 , I got:

scala> val dfComplicated = sc.parallelize(List((Map("1" -> "a"), List("b",
"c")), (Map("2" -> "b"), List("d", "e")))).toDF
...
dfComplicated: org.apache.spark.sql.DataFrame = [_1: map<string,string>,
_2: array<string>]

scala> dfComplicated.show
java.lang.OutOfMemoryError: Java heap space
  at org.apache.spark.unsafe.types.UTF8String.getBytes(UTF8String.java:229)
  at org.apache.spark.unsafe.types.UTF8String.toString(UTF8String.java:821)
  at
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificSafeProjection.apply(Unknown
Source)
  at
org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:241)
  at
org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
  at
org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
  at
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at
scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
  at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
  at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
  at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186)
  at
org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2121)
  at
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:54)
  at org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2408)
  at org.apache.spark.sql.Dataset.org
$apache$spark$sql$Dataset$$execute$1(Dataset.scala:2120)
  at org.apache.spark.sql.Dataset.org
$apache$spark$sql$Dataset$$collect(Dataset.scala:2127)
  at org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:1861)
  at org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:1860)
  at org.apache.spark.sql.Dataset.withTypedCallback(Dataset.scala:2438)
  at org.apache.spark.sql.Dataset.head(Dataset.scala:1860)
  at org.apache.spark.sql.Dataset.take(Dataset.scala:2077)
  at org.apache.spark.sql.Dataset.showString(Dataset.scala:238)
  at org.apache.spark.sql.Dataset.show(Dataset.scala:529)
  at org.apache.spark.sql.Dataset.show(Dataset.scala:489)
  at org.apache.spark.sql.Dataset.show(Dataset.scala:498)
  ... 6 elided

scala>

On Sun, May 1, 2016 at 8:34 PM, Koert Kuipers <ko...@tresata.com> wrote:

> by removing dependencies it turns into a different error, see below.
> the test is simply writing a DataFrame out to file and reading it back in.
> i see the error for all data sources (json, parquet, etc.).
>
> this is the data that i write out and read back in:
> val dfComplicated = sc.parallelize(List((Map("1" -> "a"), List("b", "c")),
> (Map("2" -> "b"), List("d", "e")))).toDF
>
>
> [info]   java.lang.RuntimeException: Error while decoding:
> java.lang.NegativeArraySizeException
> [info] createexternalrow(if (isnull(input[0, map<string,string>])) null
> else staticinvoke(class
> org.apache.spark.sql.catalyst.util.ArrayBasedMapData$, ObjectType(interface
> scala.collection.Map), toScalaMap, staticinvoke(class
> scala.collection.mutable.WrappedArray$, ObjectType(interface
> scala.collection.Seq), make,
> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
> StringType), lambdavariable(MapObjects_loopValue16,
> MapObjects_loopIsNull17, StringType).toString, input[0,
> map<string,string>].keyArray).array, true), staticinvoke(class
> scala.collection.mutable.WrappedArray$, ObjectType(interface
> scala.collection.Seq), make,
> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
> StringType), lambdavariable(MapObjects_loopValue18,
> MapObjects_loopIsNull19, StringType).toString, input[0,
> map<string,string>].valueArray).array, true), true), if (isnull(input[1,
> array<string>])) null else staticinvoke(class
> scala.collection.mutable.WrappedArray$, ObjectType(interface
> scala.collection.Seq), make,
> mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
> StringType), lambdavariable(MapObjects_loopValue20,
> MapObjects_loopIsNull21, StringType).toString, input[1,
> array<string>]).array, true),
> StructField(_1,MapType(StringType,StringType,true),true),
> StructField(_2,ArrayType(StringType,true),true))
> [info] :- if (isnull(input[0, map<string,string>])) null else
> staticinvoke(class org.apache.spark.sql.catalyst.util.ArrayBasedMapData$,
> ObjectType(interface scala.collection.Map), toScalaMap, staticinvoke(class
> scala.collection.mutable.WrappedArray$, ObjectType(interface
> scala.collection.Seq), make,
> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
> StringType), lambdavariable(MapObjects_loopValue16,
> MapObjects_loopIsNull17, StringType).toString, input[0,
> map<string,string>].keyArray).array, true), staticinvoke(class
> scala.collection.mutable.WrappedArray$, ObjectType(interface
> scala.collection.Seq), make,
> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
> StringType), lambdavariable(MapObjects_loopValue18,
> MapObjects_loopIsNull19, StringType).toString, input[0,
> map<string,string>].valueArray).array, true), true)
> [info] :  :- isnull(input[0, map<string,string>])
> [info] :  :  +- input[0, map<string,string>]
> [info] :  :- null
> [info] :  +- staticinvoke(class
> org.apache.spark.sql.catalyst.util.ArrayBasedMapData$, ObjectType(interface
> scala.collection.Map), toScalaMap, staticinvoke(class
> scala.collection.mutable.WrappedArray$, ObjectType(interface
> scala.collection.Seq), make,
> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
> StringType), lambdavariable(MapObjects_loopValue16,
> MapObjects_loopIsNull17, StringType).toString, input[0,
> map<string,string>].keyArray).array, true), staticinvoke(class
> scala.collection.mutable.WrappedArray$, ObjectType(interface
> scala.collection.Seq), make,
> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
> StringType), lambdavariable(MapObjects_loopValue18,
> MapObjects_loopIsNull19, StringType).toString, input[0,
> map<string,string>].valueArray).array, true), true)
> [info] :     :- staticinvoke(class scala.collection.mutable.WrappedArray$,
> ObjectType(interface scala.collection.Seq), make,
> mapobjects(lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
> StringType), lambdavariable(MapObjects_loopValue16,
> MapObjects_loopIsNull17, StringType).toString, input[0,
> map<string,string>].keyArray).array, true)
> [info] :     :  +- mapobjects(lambdavariable(MapObjects_loopValue16,
> MapObjects_loopIsNull17, StringType),
> lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
> StringType).toString, input[0, map<string,string>].keyArray).array
> [info] :     :     +- mapobjects(lambdavariable(MapObjects_loopValue16,
> MapObjects_loopIsNull17, StringType),
> lambdavariable(MapObjects_loopValue16, MapObjects_loopIsNull17,
> StringType).toString, input[0, map<string,string>].keyArray)
> [info] :     :        :- lambdavariable(MapObjects_loopValue16,
> MapObjects_loopIsNull17, StringType).toString
> [info] :     :        :  +- lambdavariable(MapObjects_loopValue16,
> MapObjects_loopIsNull17, StringType)
> [info] :     :        +- input[0, map<string,string>].keyArray
> [info] :     :           +- input[0, map<string,string>]
> [info] :     +- staticinvoke(class scala.collection.mutable.WrappedArray$,
> ObjectType(interface scala.collection.Seq), make,
> mapobjects(lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
> StringType), lambdavariable(MapObjects_loopValue18,
> MapObjects_loopIsNull19, StringType).toString, input[0,
> map<string,string>].valueArray).array, true)
> [info] :        +- mapobjects(lambdavariable(MapObjects_loopValue18,
> MapObjects_loopIsNull19, StringType),
> lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
> StringType).toString, input[0, map<string,string>].valueArray).array
> [info] :           +- mapobjects(lambdavariable(MapObjects_loopValue18,
> MapObjects_loopIsNull19, StringType),
> lambdavariable(MapObjects_loopValue18, MapObjects_loopIsNull19,
> StringType).toString, input[0, map<string,string>].valueArray)
> [info] :              :- lambdavariable(MapObjects_loopValue18,
> MapObjects_loopIsNull19, StringType).toString
> [info] :              :  +- lambdavariable(MapObjects_loopValue18,
> MapObjects_loopIsNull19, StringType)
> [info] :              +- input[0, map<string,string>].valueArray
> [info] :                 +- input[0, map<string,string>]
> [info] +- if (isnull(input[1, array<string>])) null else
> staticinvoke(class scala.collection.mutable.WrappedArray$,
> ObjectType(interface scala.collection.Seq), make,
> mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
> StringType), lambdavariable(MapObjects_loopValue20,
> MapObjects_loopIsNull21, StringType).toString, input[1,
> array<string>]).array, true)
> [info]    :- isnull(input[1, array<string>])
> [info]    :  +- input[1, array<string>]
> [info]    :- null
> [info]    +- staticinvoke(class scala.collection.mutable.WrappedArray$,
> ObjectType(interface scala.collection.Seq), make,
> mapobjects(lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
> StringType), lambdavariable(MapObjects_loopValue20,
> MapObjects_loopIsNull21, StringType).toString, input[1,
> array<string>]).array, true)
> [info]       +- mapobjects(lambdavariable(MapObjects_loopValue20,
> MapObjects_loopIsNull21, StringType),
> lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
> StringType).toString, input[1, array<string>]).array
> [info]          +- mapobjects(lambdavariable(MapObjects_loopValue20,
> MapObjects_loopIsNull21, StringType),
> lambdavariable(MapObjects_loopValue20, MapObjects_loopIsNull21,
> StringType).toString, input[1, array<string>])
> [info]             :- lambdavariable(MapObjects_loopValue20,
> MapObjects_loopIsNull21, StringType).toString
> [info]             :  +- lambdavariable(MapObjects_loopValue20,
> MapObjects_loopIsNull21, StringType)
> [info]             +- input[1, array<string>]
> [info]   at
> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:244)
> [info]   at
> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
> [info]   at
> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Dataset.scala:2121)
> [info]   at
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
> [info]   at
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
> [info]   at
> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
> [info]   at
> scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
> [info]   at
> scala.collection.TraversableLike$class.map(TraversableLike.scala:245)
> [info]   at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186)
> [info]   at
> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2121)
> [info]   at
> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:54)
> [info]   at
> org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2408)
> [info]   at org.apache.spark.sql.Dataset.org
> $apache$spark$sql$Dataset$$execute$1(Dataset.scala:2120)
> [info]   at
> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Dataset.scala:2125)
> [info]   at
> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Dataset.scala:2125)
> [info]   at org.apache.spark.sql.Dataset.withCallback(Dataset.scala:2421)
> [info]   at org.apache.spark.sql.Dataset.org
> $apache$spark$sql$Dataset$$collect(Dataset.scala:2125)
> [info]   at org.apache.spark.sql.Dataset.collect(Dataset.scala:2101)
>
>
>
> On Thu, Apr 28, 2016 at 2:41 PM, Ted Yu <yu...@gmail.com> wrote:
>
>> Are you able to pastebin a unit test which can reproduce the following ?
>>
>> Thanks
>>
>> On Apr 28, 2016, at 11:35 AM, Koert Kuipers <ko...@tresata.com> wrote:
>>
>> i tried for the first time to run our own in-house unit tests on spark 2,
>> and i get the error below.
>> has anyone seen this?
>>
>> it is reproducible. i tried latest java 7 and it is still there.
>>
>> # A fatal error has been detected by the Java Runtime Environment:
>> #
>> #  SIGSEGV (0xb) at pc=0x00007f7c3a4b1f54, pid=21939, tid=140171011417856
>> #
>> # JRE version: Java(TM) SE Runtime Environment (7.0_75-b13) (build
>> 1.7.0_75-b13)
>> # Java VM: Java HotSpot(TM) 64-Bit Server VM (24.75-b04 mixed mode
>> linux-amd64 compressed oops)
>> # Problematic frame:
>> # V  [libjvm.so+0x747f54]  _Copy_arrayof_conjoint_jlongs+0x44
>>
>> more info:
>>
>> Stack: [0x00007f7c1b47e000,0x00007f7c1b57f000],  sp=0x00007f7c1b57a9a8,
>> free space=1010k
>> Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native
>> code)
>> V  [libjvm.so+0x747f54]  _Copy_arrayof_conjoint_jlongs+0x44
>> j  sun.misc.Unsafe.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V+0
>> j
>> org.apache.spark.unsafe.Platform.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V+34
>> j  org.apache.spark.unsafe.types.UTF8String.getBytes()[B+76
>> j  org.apache.spark.unsafe.types.UTF8String.toString()Ljava/lang/String;+5
>> j
>> org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificSafeProjection.apply(Ljava/lang/Object;)Ljava/lang/Object;+876
>> j
>> org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(Lorg/apache/spark/sql/catalyst/InternalRow;)Ljava/lang/Object;+5
>> j
>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Lorg/apache/spark/sql/catalyst/InternalRow;)Ljava/lang/Object;+11
>> j
>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Ljava/lang/Object;)Ljava/lang/Object;+5
>> J 13277 C2
>> scala.collection.mutable.ArrayOps$ofRef.map(Lscala/Function1;Lscala/collection/generic/CanBuildFrom;)Ljava/lang/Object;
>> (7 bytes) @ 0x00007f7c25eeae08 [0x00007f7c25eead40+0xc8]
>> j
>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply()Ljava/lang/Object;+43
>> j
>> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(Lorg/apache/spark/sql/SparkSession;Lorg/apache/spark/sql/execution/QueryExecution;Lscala/Function0;)Ljava/lang/Object;+106
>> j
>> org.apache.spark.sql.Dataset.withNewExecutionId(Lscala/Function0;)Ljava/lang/Object;+12
>> j  org.apache.spark.sql.Dataset.org
>> $apache$spark$sql$Dataset$$execute$1()Ljava/lang/Object;+9
>> j
>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Lorg/apache/spark/sql/Dataset;)Ljava/lang/Object;+4
>> j
>> org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Ljava/lang/Object;)Ljava/lang/Object;+5
>> j
>> org.apache.spark.sql.Dataset.withCallback(Ljava/lang/String;Lorg/apache/spark/sql/Dataset;Lscala/Function1;)Ljava/lang/Object;+25
>> j  org.apache.spark.sql.Dataset.org
>> $apache$spark$sql$Dataset$$collect(Z)Ljava/lang/Object;+20
>> j  org.apache.spark.sql.Dataset.collect()Ljava/lang/Object;+2
>>
>>
>

Re: spark 2 segfault

Posted by Ted Yu <yu...@gmail.com>.
Are you able to pastebin a unit test which can reproduce the following ?

Thanks

> On Apr 28, 2016, at 11:35 AM, Koert Kuipers <ko...@tresata.com> wrote:
> 
> i tried for the first time to run our own in-house unit tests on spark 2, and i get the error below.
> has anyone seen this?
> 
> it is reproducible. i tried latest java 7 and it is still there.
> 
> # A fatal error has been detected by the Java Runtime Environment:
> #
> #  SIGSEGV (0xb) at pc=0x00007f7c3a4b1f54, pid=21939, tid=140171011417856
> #
> # JRE version: Java(TM) SE Runtime Environment (7.0_75-b13) (build 1.7.0_75-b13)
> # Java VM: Java HotSpot(TM) 64-Bit Server VM (24.75-b04 mixed mode linux-amd64 compressed oops)
> # Problematic frame:
> # V  [libjvm.so+0x747f54]  _Copy_arrayof_conjoint_jlongs+0x44
> 
> more info:
> 
> Stack: [0x00007f7c1b47e000,0x00007f7c1b57f000],  sp=0x00007f7c1b57a9a8,  free space=1010k
> Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code)
> V  [libjvm.so+0x747f54]  _Copy_arrayof_conjoint_jlongs+0x44
> j  sun.misc.Unsafe.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V+0
> j  org.apache.spark.unsafe.Platform.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V+34
> j  org.apache.spark.unsafe.types.UTF8String.getBytes()[B+76
> j  org.apache.spark.unsafe.types.UTF8String.toString()Ljava/lang/String;+5
> j  org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificSafeProjection.apply(Ljava/lang/Object;)Ljava/lang/Object;+876
> j  org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(Lorg/apache/spark/sql/catalyst/InternalRow;)Ljava/lang/Object;+5
> j  org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Lorg/apache/spark/sql/catalyst/InternalRow;)Ljava/lang/Object;+11
> j  org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$13.apply(Ljava/lang/Object;)Ljava/lang/Object;+5
> J 13277 C2 scala.collection.mutable.ArrayOps$ofRef.map(Lscala/Function1;Lscala/collection/generic/CanBuildFrom;)Ljava/lang/Object; (7 bytes) @ 0x00007f7c25eeae08 [0x00007f7c25eead40+0xc8]
> j  org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply()Ljava/lang/Object;+43
> j  org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(Lorg/apache/spark/sql/SparkSession;Lorg/apache/spark/sql/execution/QueryExecution;Lscala/Function0;)Ljava/lang/Object;+106
> j  org.apache.spark.sql.Dataset.withNewExecutionId(Lscala/Function0;)Ljava/lang/Object;+12
> j  org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$execute$1()Ljava/lang/Object;+9
> j  org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Lorg/apache/spark/sql/Dataset;)Ljava/lang/Object;+4
> j  org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Ljava/lang/Object;)Ljava/lang/Object;+5
> j  org.apache.spark.sql.Dataset.withCallback(Ljava/lang/String;Lorg/apache/spark/sql/Dataset;Lscala/Function1;)Ljava/lang/Object;+25
> j  org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$collect(Z)Ljava/lang/Object;+20
> j  org.apache.spark.sql.Dataset.collect()Ljava/lang/Object;+2
>