[ 
https://issues.apache.org/jira/browse/SPARK-17403?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15982595#comment-15982595
 ] 

Paul Lysak edited comment on SPARK-17403 at 4/26/17 3:23 PM:
-------------------------------------------------------------

Looks like we have the same issue with Spark 2.1 on YARN (Amazon EMR release 
emr-5.4.0).
Workaround that solves the issue for us (at the cost of some performance) is to 
use df.persist(StorageLevel.DISK_ONLY) instead of df.cache().

Depending on the node types, memory settings, storage level and some other 
factors I couldn't clearly identify it may appear as 

{noformat}
User class threw exception: org.apache.spark.SparkException: Job aborted due to 
stage failure: Task 158 in stage 504.0 failed 4 times, most recent failure: 
Lost task 158.3 in stage 504.0 (TID 427365, ip-10-35-162-171.ec2.internal, 
executor 83): java.lang.NegativeArraySizeException
at org.apache.spark.unsafe.types.UTF8String.getBytes(UTF8String.java:229)
at org.apache.spark.unsafe.types.UTF8String.clone(UTF8String.java:826)
at 
org.apache.spark.sql.execution.columnar.StringColumnStats.gatherStats(ColumnStats.scala:217)
at 
org.apache.spark.sql.execution.columnar.NullableColumnBuilder$class.appendFrom(NullableColumnBuilder.scala:55)
at 
org.apache.spark.sql.execution.columnar.NativeColumnBuilder.org$apache$spark$sql$execution$columnar$compression$CompressibleColumnBuilder$$super$appendFrom(ColumnBuilder.scala:97)
at 
org.apache.spark.sql.execution.columnar.compression.CompressibleColumnBuilder$class.appendFrom(CompressibleColumnBuilder.scala:78)
at 
org.apache.spark.sql.execution.columnar.NativeColumnBuilder.appendFrom(ColumnBuilder.scala:97)
at 
org.apache.spark.sql.execution.columnar.InMemoryRelation$$anonfun$1$$anon$1.next(InMemoryRelation.scala:122)
at 
org.apache.spark.sql.execution.columnar.InMemoryRelation$$anonfun$1$$anon$1.next(InMemoryRelation.scala:97)
at 
org.apache.spark.storage.memory.MemoryStore.putIteratorAsValues(MemoryStore.scala:216)
at 
org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:957)
at 
org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:948)
at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:888)
at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:948)
at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:694)
at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:334)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:285)
{noformat}

or as 

{noformat}
User class threw exception: org.apache.spark.SparkException: Job aborted due to 
stage failure: Task 27 in stage 61.0 failed 4 times, most recent failure: Lost 
task 27.3 in stage 61.0 (TID 36167, ip-10-35-162-149.ec2.internal, executor 1): 
java.lang.OutOfMemoryError: Java heap space
at 
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder.grow(BufferHolder.java:73)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply_38$(Unknown
 Source)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown
 Source)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown
 Source)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
at 
org.apache.spark.sql.execution.columnar.InMemoryRelation$$anonfun$1$$anon$1.next(InMemoryRelation.scala:107)
at 
org.apache.spark.sql.execution.columnar.InMemoryRelation$$anonfun$1$$anon$1.next(InMemoryRelation.scala:97)
at 
org.apache.spark.storage.memory.MemoryStore.putIteratorAsValues(MemoryStore.scala:216)
at 
org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:957)
at 
org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:948)
at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:888)
at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:948)
at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:694)
at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:334)
{noformat}

or as

{noformat}
2017-04-24 19:02:45,951 ERROR 
org.apache.spark.util.SparkUncaughtExceptionHandler: Uncaught exception in 
thread Thread[Executor task launch worker-3,5,main]
java.lang.OutOfMemoryError: Requested array size exceeds VM limit
        at 
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder.grow(BufferHolder.java:73)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply_37$(Unknown
 Source)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown
 Source)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown
 Source)
        at 
org.apache.spark.sql.execution.joins.HashJoin$$anonfun$join$1.apply(HashJoin.scala:217)
        at 
org.apache.spark.sql.execution.joins.HashJoin$$anonfun$join$1.apply(HashJoin.scala:215)
        at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
        at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
        at 
org.apache.spark.sql.execution.columnar.InMemoryRelation$$anonfun$1$$anon$1.next(InMemoryRelation.scala:107)
        at 
org.apache.spark.sql.execution.columnar.InMemoryRelation$$anonfun$1$$anon$1.next(InMemoryRelation.scala:97)
        at 
org.apache.spark.storage.memory.MemoryStore.putIteratorAsValues(MemoryStore.scala:216)
        at 
org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:957)
        at 
org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:948)
        at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:888)
        at 
org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:948)
        at 
org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:694)
        at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:334)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:285)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
{noformat}

With the same input data, node types, settings and code error is deterministic 
- it always fails in the same place with the same stacktrace. 
We have quite a few joins with cached inputs, time to time we do checkpoints. 
If we comment out all caching between the checkpoint and failing join - this 
specific join works, the next (where we didn't comment out the caching)
however fails. 
The problem starts to appear when input of the application becomes big enough - 
I've seen the error happening when stage input was 100+ mln records taking 400+ 
GB size on 20 c4.xlarge nodes. Using nodes with more RAM lets application
go a bit further and fail a bit later, but eventually we still hit the issue.
As I said, application manages to complete successfully if we persist data 
frames with DISK_ONLY storage level.




was (Author: paul_lysak):
Looks like we have the same issue with Spark 2.1 on YARN (Amazon EMR release 
emr-5.4.0).
Workaround that solves the issue for us (at the cost of some performance) is to 
use df.persist(StorageLevel.DISK_ONLY) instead of df.cache().

Depending on the node types, memory settings, storage level and some other 
factors I couldn't clearly identify it may appear as 

User class threw exception: org.apache.spark.SparkException: Job aborted due to 
stage failure: Task 158 in stage 504.0 failed 4 times, most recent failure: 
Lost task 158.3 in stage 504.0 (TID 427365, ip-10-35-162-171.ec2.internal, 
executor 83): java.lang.NegativeArraySizeException
at org.apache.spark.unsafe.types.UTF8String.getBytes(UTF8String.java:229)
at org.apache.spark.unsafe.types.UTF8String.clone(UTF8String.java:826)
at 
org.apache.spark.sql.execution.columnar.StringColumnStats.gatherStats(ColumnStats.scala:217)
at 
org.apache.spark.sql.execution.columnar.NullableColumnBuilder$class.appendFrom(NullableColumnBuilder.scala:55)
at 
org.apache.spark.sql.execution.columnar.NativeColumnBuilder.org$apache$spark$sql$execution$columnar$compression$CompressibleColumnBuilder$$super$appendFrom(ColumnBuilder.scala:97)
at 
org.apache.spark.sql.execution.columnar.compression.CompressibleColumnBuilder$class.appendFrom(CompressibleColumnBuilder.scala:78)
at 
org.apache.spark.sql.execution.columnar.NativeColumnBuilder.appendFrom(ColumnBuilder.scala:97)
at 
org.apache.spark.sql.execution.columnar.InMemoryRelation$$anonfun$1$$anon$1.next(InMemoryRelation.scala:122)
at 
org.apache.spark.sql.execution.columnar.InMemoryRelation$$anonfun$1$$anon$1.next(InMemoryRelation.scala:97)
at 
org.apache.spark.storage.memory.MemoryStore.putIteratorAsValues(MemoryStore.scala:216)
at 
org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:957)
at 
org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:948)
at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:888)
at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:948)
at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:694)
at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:334)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:285)

or as 

User class threw exception: org.apache.spark.SparkException: Job aborted due to 
stage failure: Task 27 in stage 61.0 failed 4 times, most recent failure: Lost 
task 27.3 in stage 61.0 (TID 36167, ip-10-35-162-149.ec2.internal, executor 1): 
java.lang.OutOfMemoryError: Java heap space
at 
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder.grow(BufferHolder.java:73)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply_38$(Unknown
 Source)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown
 Source)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown
 Source)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
at 
org.apache.spark.sql.execution.columnar.InMemoryRelation$$anonfun$1$$anon$1.next(InMemoryRelation.scala:107)
at 
org.apache.spark.sql.execution.columnar.InMemoryRelation$$anonfun$1$$anon$1.next(InMemoryRelation.scala:97)
at 
org.apache.spark.storage.memory.MemoryStore.putIteratorAsValues(MemoryStore.scala:216)
at 
org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:957)
at 
org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:948)
at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:888)
at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:948)
at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:694)
at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:334)

or as

2017-04-24 19:02:45,951 ERROR 
org.apache.spark.util.SparkUncaughtExceptionHandler: Uncaught exception in 
thread Thread[Executor task launch worker-3,5,main]
java.lang.OutOfMemoryError: Requested array size exceeds VM limit
        at 
org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder.grow(BufferHolder.java:73)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply_37$(Unknown
 Source)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown
 Source)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown
 Source)
        at 
org.apache.spark.sql.execution.joins.HashJoin$$anonfun$join$1.apply(HashJoin.scala:217)
        at 
org.apache.spark.sql.execution.joins.HashJoin$$anonfun$join$1.apply(HashJoin.scala:215)
        at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
        at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
        at 
org.apache.spark.sql.execution.columnar.InMemoryRelation$$anonfun$1$$anon$1.next(InMemoryRelation.scala:107)
        at 
org.apache.spark.sql.execution.columnar.InMemoryRelation$$anonfun$1$$anon$1.next(InMemoryRelation.scala:97)
        at 
org.apache.spark.storage.memory.MemoryStore.putIteratorAsValues(MemoryStore.scala:216)
        at 
org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:957)
        at 
org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:948)
        at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:888)
        at 
org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:948)
        at 
org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:694)
        at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:334)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:285)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)


With the same input data, node types, settings and code error is deterministic 
- it always fails in the same place with the same stacktrace. 
We have quite a few joins with cached inputs, time to time we do checkpoints. 
If we comment out all caching between the checkpoint and failing join - this 
specific join works, the next (where we didn't comment out the caching)
however fails. 
The problem starts to appear when input of the application becomes big enough - 
I've seen the error happening when stage input was 100+ mln records taking 400+ 
GB size on 20 c4.xlarge nodes. Using nodes with more RAM lets application
go a bit further and fail a bit later, but eventually we still hit the issue.
As I said, application manages to complete successfully if we persist data 
frames with DISK_ONLY storage level.



> Fatal Error: Scan cached strings
> --------------------------------
>
>                 Key: SPARK-17403
>                 URL: https://issues.apache.org/jira/browse/SPARK-17403
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.0.0
>         Environment: Spark standalone cluster (3 Workers, 47 cores)
> Ubuntu 14
> Java 8
>            Reporter: Ruben Hernando
>
> The process creates views from JDBC (SQL server) source and combines them to 
> create other views.
> Finally it dumps results via JDBC
> Error:
> {quote}
> # JRE version: Java(TM) SE Runtime Environment (8.0_101-b13) (build 
> 1.8.0_101-b13)
> # Java VM: Java HotSpot(TM) 64-Bit Server VM (25.101-b13 mixed mode 
> linux-amd64 )
> # Problematic frame:
> # J 4895 C1 org.apache.spark.unsafe.Platform.getLong(Ljava/lang/Object;J)J (9 
> bytes) @ 0x00007fbb355dfd6c [0x00007fbb355dfd60+0xc]
> #
> {quote}
> SQL Query plan (fields truncated):
> {noformat}
> == Parsed Logical Plan ==
> 'Project [*]
> +- 'UnresolvedRelation `COEQ_63`
> == Analyzed Logical Plan ==
> InstanceId: bigint, price: double, ZoneId: int, priceItemId: int, priceId: int
> Project [InstanceId#20236L, price#20237, ZoneId#20239, priceItemId#20242, 
> priceId#20244]
> +- SubqueryAlias coeq_63
>    +- Project [_TableSL_SID#143L AS InstanceId#20236L, SL_RD_ColR_N#189 AS 
> price#20237, 24 AS ZoneId#20239, 6 AS priceItemId#20242, 63 AS priceId#20244]
>       +- SubqueryAlias 6__input
>          +- 
> Relation[_TableSL_SID#143L,_TableP_DC_SID#144L,_TableSH_SID#145L,ID#146,Name#147,TableP_DCID#148,TableSHID#149,SL_ACT_GI_DTE#150,SL_Xcl_C#151,SL_Xcl_C#152,SL_Css_Cojs#153L,SL_Config#154,SL_CREATEDON#
>  .......... 36 more fields] JDBCRelation((select [SLTables].[_TableSL_SID], 
> [SLTables]. ... [...]  FROM [sch].[SLTables] [SLTables] JOIN sch.TPSLTables 
> TPSLTables ON [TPSLTables].[_TableSL_SID] = [SLTables].[_TableSL_SID] where 
> _TP = 24) input)
> == Optimized Logical Plan ==
> Project [_TableSL_SID#143L AS InstanceId#20236L, SL_RD_ColR_N#189 AS 
> price#20237, 24 AS ZoneId#20239, 6 AS priceItemId#20242, 63 AS priceId#20244]
> +- InMemoryRelation [_TableSL_SID#143L, _TableP_DC_SID#144L, 
> _TableSH_SID#145L, ID#146, Name#147, ... 36 more fields], true, 10000, 
> StorageLevel(disk, memory, deserialized, 1 replicas)
>    :  +- *Scan JDBCRelation((select [SLTables].[_TableSL_SID], 
> [SLTables].[_TableP_DC_SID], [SLTables].[_TableSH_SID], [SLTables].[ID], 
> [SLTables].[Name], [SLTables].[TableP_DCID], [SLTables].[TableSHID], 
> [TPSLTables].[SL_ACT_GI_DTE],  ... [...] FROM [sch].[SLTables] [SLTables] 
> JOIN sch.TPSLTables TPSLTables ON [TPSLTables].[_TableSL_SID] = 
> [SLTables].[_TableSL_SID] where _TP = 24) input) 
> [_TableSL_SID#143L,_TableP_DC_SID#144L,_TableSH_SID#145L,ID#146,Name#147,TableP_DCID#148,TableSHID#149,SL_ACT_GI_DTE#150,SL_Xcl_C#151,...
>  36 more fields] 
> == Physical Plan ==
> *Project [_TableSL_SID#143L AS InstanceId#20236L, SL_RD_ColR_N#189 AS 
> price#20237, 24 AS ZoneId#20239, 6 AS priceItemId#20242, 63 AS priceId#20244]
> +- InMemoryTableScan [_TableSL_SID#143L, SL_RD_ColR_N#189]
>    :  +- InMemoryRelation [_TableSL_SID#143L, _TableP_DC_SID#144L, 
> _TableSH_SID#145L, ID#146, Name#147, ... 36 more fields], true, 10000, 
> StorageLevel(disk, memory, deserialized, 1 replicas)
>    :     :  +- *Scan JDBCRelation((select [SLTables].[_TableSL_SID], 
> [SLTables].[_TableP_DC_SID], [SLTables].[_TableSH_SID], [SLTables].[ID], 
> [SLTables].[Name], [SLTables].[TableP_DCID],  ... [...] FROM [sch].[SLTables] 
> [SLTables] JOIN sch.TPSLTables TPSLTables ON [TPSLTables].[_TableSL_SID] = 
> [SLTables].[_TableSL_SID] where _TP = 24) input) 
> [_TableSL_SID#143L,_TableP_DC_SID#144L,_TableSH_SID#145L,ID#146,Name#147,,... 
> 36 more fields]
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

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

Reply via email to