You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@spark.apache.org by Pete Robbins <ro...@gmail.com> on 2016/06/08 14:24:31 UTC
Re: NegativeArraySizeException / segfault
I just raised https://issues.apache.org/jira/browse/SPARK-15822 for a
similar looking issue. Analyzing the core dump from the segv with Memory
Analyzer it looks very much like a UTF8String is very corrupt.
Cheers,
On Fri, 27 May 2016 at 21:00 Koert Kuipers <ko...@tresata.com> wrote:
> hello all,
> after getting our unit tests to pass on spark 2.0.0-SNAPSHOT we are now
> trying to run some algorithms at scale on our cluster.
> unfortunately this means that when i see errors i am having a harder time
> boiling it down to a small reproducible example.
>
> today we are running an iterative algo using the dataset api and we are
> seeing tasks fail with errors which seem to related to unsafe operations.
> the same tasks succeed without issues in our unit tests.
>
> i see either:
>
> 16/05/27 12:54:46 ERROR executor.Executor: Exception in task 31.0 in stage
> 21.0 (TID 1073)
> java.lang.NegativeArraySizeException
> 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 scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
> at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
> at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
> at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
> at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
> at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
> at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
> at
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.sort_addToSorter$(Unknown
> Source)
> at
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown
> Source)
> at
> org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
> at
> org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$7$$anon$1.hasNext(WholeStageCodegenExec.scala:359)
> at
> org.apache.spark.sql.execution.aggregate.SortBasedAggregateExec$$anonfun$doExecute$1$$anonfun$3.apply(SortBasedAggregateExec.scala:74)
> at
> org.apache.spark.sql.execution.aggregate.SortBasedAggregateExec$$anonfun$doExecute$1$$anonfun$3.apply(SortBasedAggregateExec.scala:71)
> at
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:775)
> at
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:775)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:282)
> at
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:282)
> at
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:79)
> at
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:47)
> at org.apache.spark.scheduler.Task.run(Task.scala:85)
> at
> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>
> or alternatively:
>
> # A fatal error has been detected by the Java Runtime Environment:
> #
> # SIGSEGV (0xb) at pc=0x00007fe571041cba, pid=2450, tid=140622965913344
> #
> # 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 ~StubRoutines::jbyte_disjoint_arraycopy
>
> i assume the best thing would be to try to get it to print out the
> generated code that is causing this?
> what switch do i need to use again to do so?
> thanks,
> koert
>
Re: NegativeArraySizeException / segfault
Posted by Andres Perez <an...@tresata.com>.
We were able to reproduce it with a minimal example. I've opened a jira
issue:
https://issues.apache.org/jira/browse/SPARK-15825
On Wed, Jun 8, 2016 at 12:43 PM, Koert Kuipers <ko...@tresata.com> wrote:
> great!
>
> we weren't able to reproduce it because the unit tests use a
> broadcast-join while on the cluster it uses sort-merge-join. so the issue
> is in sort-merge-join.
>
> we are now able to reproduce it in tests using
> spark.sql.autoBroadcastJoinThreshold=-1
> it produces weird looking garbled results in the join.
> hoping to get a minimal reproducible example soon.
>
> On Wed, Jun 8, 2016 at 10:24 AM, Pete Robbins <ro...@gmail.com> wrote:
>
>> I just raised https://issues.apache.org/jira/browse/SPARK-15822 for a
>> similar looking issue. Analyzing the core dump from the segv with Memory
>> Analyzer it looks very much like a UTF8String is very corrupt.
>>
>> Cheers,
>>
>>
>> On Fri, 27 May 2016 at 21:00 Koert Kuipers <ko...@tresata.com> wrote:
>>
>>> hello all,
>>> after getting our unit tests to pass on spark 2.0.0-SNAPSHOT we are now
>>> trying to run some algorithms at scale on our cluster.
>>> unfortunately this means that when i see errors i am having a harder
>>> time boiling it down to a small reproducible example.
>>>
>>> today we are running an iterative algo using the dataset api and we are
>>> seeing tasks fail with errors which seem to related to unsafe operations.
>>> the same tasks succeed without issues in our unit tests.
>>>
>>> i see either:
>>>
>>> 16/05/27 12:54:46 ERROR executor.Executor: Exception in task 31.0 in
>>> stage 21.0 (TID 1073)
>>> java.lang.NegativeArraySizeException
>>> 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 scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
>>> at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
>>> at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
>>> at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
>>> at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
>>> at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
>>> at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
>>> at
>>> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.sort_addToSorter$(Unknown
>>> Source)
>>> at
>>> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown
>>> Source)
>>> at
>>> org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
>>> at
>>> org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$7$$anon$1.hasNext(WholeStageCodegenExec.scala:359)
>>> at
>>> org.apache.spark.sql.execution.aggregate.SortBasedAggregateExec$$anonfun$doExecute$1$$anonfun$3.apply(SortBasedAggregateExec.scala:74)
>>> at
>>> org.apache.spark.sql.execution.aggregate.SortBasedAggregateExec$$anonfun$doExecute$1$$anonfun$3.apply(SortBasedAggregateExec.scala:71)
>>> at
>>> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:775)
>>> at
>>> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:775)
>>> at
>>> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>>> at
>>> org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318)
>>> at org.apache.spark.rdd.RDD.iterator(RDD.scala:282)
>>> at
>>> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>>> at
>>> org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318)
>>> at org.apache.spark.rdd.RDD.iterator(RDD.scala:282)
>>> at
>>> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:79)
>>> at
>>> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:47)
>>> at org.apache.spark.scheduler.Task.run(Task.scala:85)
>>> at
>>> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
>>> at
>>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>>> at
>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>>>
>>> or alternatively:
>>>
>>> # A fatal error has been detected by the Java Runtime Environment:
>>> #
>>> # SIGSEGV (0xb) at pc=0x00007fe571041cba, pid=2450, tid=140622965913344
>>> #
>>> # 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 ~StubRoutines::jbyte_disjoint_arraycopy
>>>
>>> i assume the best thing would be to try to get it to print out the
>>> generated code that is causing this?
>>> what switch do i need to use again to do so?
>>> thanks,
>>> koert
>>>
>>
>
Re: NegativeArraySizeException / segfault
Posted by Koert Kuipers <ko...@tresata.com>.
great!
we weren't able to reproduce it because the unit tests use a broadcast-join
while on the cluster it uses sort-merge-join. so the issue is in
sort-merge-join.
we are now able to reproduce it in tests using
spark.sql.autoBroadcastJoinThreshold=-1
it produces weird looking garbled results in the join.
hoping to get a minimal reproducible example soon.
On Wed, Jun 8, 2016 at 10:24 AM, Pete Robbins <ro...@gmail.com> wrote:
> I just raised https://issues.apache.org/jira/browse/SPARK-15822 for a
> similar looking issue. Analyzing the core dump from the segv with Memory
> Analyzer it looks very much like a UTF8String is very corrupt.
>
> Cheers,
>
>
> On Fri, 27 May 2016 at 21:00 Koert Kuipers <ko...@tresata.com> wrote:
>
>> hello all,
>> after getting our unit tests to pass on spark 2.0.0-SNAPSHOT we are now
>> trying to run some algorithms at scale on our cluster.
>> unfortunately this means that when i see errors i am having a harder time
>> boiling it down to a small reproducible example.
>>
>> today we are running an iterative algo using the dataset api and we are
>> seeing tasks fail with errors which seem to related to unsafe operations.
>> the same tasks succeed without issues in our unit tests.
>>
>> i see either:
>>
>> 16/05/27 12:54:46 ERROR executor.Executor: Exception in task 31.0 in
>> stage 21.0 (TID 1073)
>> java.lang.NegativeArraySizeException
>> 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 scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
>> at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
>> at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
>> at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
>> at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
>> at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
>> at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
>> at
>> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.sort_addToSorter$(Unknown
>> Source)
>> at
>> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown
>> Source)
>> at
>> org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
>> at
>> org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$7$$anon$1.hasNext(WholeStageCodegenExec.scala:359)
>> at
>> org.apache.spark.sql.execution.aggregate.SortBasedAggregateExec$$anonfun$doExecute$1$$anonfun$3.apply(SortBasedAggregateExec.scala:74)
>> at
>> org.apache.spark.sql.execution.aggregate.SortBasedAggregateExec$$anonfun$doExecute$1$$anonfun$3.apply(SortBasedAggregateExec.scala:71)
>> at
>> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:775)
>> at
>> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:775)
>> at
>> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318)
>> at org.apache.spark.rdd.RDD.iterator(RDD.scala:282)
>> at
>> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318)
>> at org.apache.spark.rdd.RDD.iterator(RDD.scala:282)
>> at
>> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:79)
>> at
>> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:47)
>> at org.apache.spark.scheduler.Task.run(Task.scala:85)
>> at
>> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
>> at
>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>> at
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>>
>> or alternatively:
>>
>> # A fatal error has been detected by the Java Runtime Environment:
>> #
>> # SIGSEGV (0xb) at pc=0x00007fe571041cba, pid=2450, tid=140622965913344
>> #
>> # 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 ~StubRoutines::jbyte_disjoint_arraycopy
>>
>> i assume the best thing would be to try to get it to print out the
>> generated code that is causing this?
>> what switch do i need to use again to do so?
>> thanks,
>> koert
>>
>