You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Eyal Farago (JIRA)" <ji...@apache.org> on 2017/07/05 08:07:00 UTC

[jira] [Commented] (SPARK-3151) DiskStore attempts to map any size BlockId without checking MappedByteBuffer limit

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

Eyal Farago commented on SPARK-3151:
------------------------------------

I just hit this with spark 2.1 when processing a disk persisted RDD
while the root cause for this is probably data skew, this seems like a severe limitation on spark.
this specific failure is a bit surprising as spark already knows the block size on disk at this point (it maps the entire file from offset 0 to file size), so it should easily be possible to split this into several blocks, after all the code is using ChunkedByteBuffer.
a better approach would be lazily loading the blocks as deserialization progresses , and an even better solution (proposed by [~matei] in [comment|https://issues.apache.org/jira/browse/SPARK-1476?focusedCommentId=13967947&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13967947] for #1476) would be splitting these block in higher levels of spark (a mapper that produces multiple blocks, cached RDD partition that consists of several blocks...)

I can see the attached PR is closed, is there an expected/in-progress fix for this?


2017-07-05 07:04:51,368 UTC	WARN 	task-result-getter-0	org.apache.spark.scheduler.TaskSetManager	 Lost task 131.0 in stage 14.0 (TID 2228, 172.20.1.137, executor 1): java.lang.IllegalArgumentException: Size exceeds Integer.MAX_VALUE
	at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:869)
	at org.apache.spark.storage.DiskStore$$anonfun$getBytes$2.apply(DiskStore.scala:103)
	at org.apache.spark.storage.DiskStore$$anonfun$getBytes$2.apply(DiskStore.scala:91)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1303)
	at org.apache.spark.storage.DiskStore.getBytes(DiskStore.scala:105)
	at org.apache.spark.storage.BlockManager.getLocalValues(BlockManager.scala:465)
	at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:701)
	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.UnionRDD.compute(UnionRDD.scala:105)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
	at org.apache.spark.scheduler.Task.run(Task.scala:99)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:748)

> DiskStore attempts to map any size BlockId without checking MappedByteBuffer limit
> ----------------------------------------------------------------------------------
>
>                 Key: SPARK-3151
>                 URL: https://issues.apache.org/jira/browse/SPARK-3151
>             Project: Spark
>          Issue Type: Bug
>          Components: Block Manager, Spark Core
>    Affects Versions: 1.0.2
>         Environment: IBM 64-bit JVM PPC64
>            Reporter: Damon Brown
>            Priority: Minor
>
> [DiskStore|https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/storage/DiskStore.scala] attempts to memory map the block file in {{def getBytes}}.  If the file is larger than 2GB (Integer.MAX_VALUE) as specified by [FileChannel.map|http://docs.oracle.com/javase/7/docs/api/java/nio/channels/FileChannel.html#map%28java.nio.channels.FileChannel.MapMode,%20long,%20long%29], then the memory map fails.
> {code}
> Some(channel.map(MapMode.READ_ONLY, segment.offset, segment.length)) # line 104
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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