You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Davies Liu (JIRA)" <ji...@apache.org> on 2015/12/02 23:49:11 UTC

[jira] [Updated] (SPARK-12089) [][]][][]][[[

     [ https://issues.apache.org/jira/browse/SPARK-12089?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Davies Liu updated SPARK-12089:
-------------------------------
    Summary: [][]][][]][[[  (was: java.lang.NegativeArraySizeException when growing BufferHolder)

> [][]][][]][[[
> -------------
>
>                 Key: SPARK-12089
>                 URL: https://issues.apache.org/jira/browse/SPARK-12089
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 1.6.0
>            Reporter: Erik Selin
>            Priority: Critical
>
> When running a large spark sql query including multiple joins I see tasks failing with the following trace:
> {code}
> java.lang.NegativeArraySizeException
>         at org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder.grow(BufferHolder.java:36)
>         at org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(UnsafeRowWriter.java:188)
>         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.OneSideOuterIterator.getRow(SortMergeOuterJoin.scala:288)
>         at org.apache.spark.sql.execution.RowIteratorToScala.next(RowIterator.scala:76)
>         at org.apache.spark.sql.execution.RowIteratorToScala.next(RowIterator.scala:62)
>         at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
>         at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:164)
>         at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
>         at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
>         at org.apache.spark.scheduler.Task.run(Task.scala:88)
>         at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>         at java.lang.Thread.run(Thread.java:745)
> {code}
> From the spark code it looks like this is due to a integer overflow when growing a buffer length. The offending line {{BufferHolder.java:36}} is the following in the version I'm running:
> {code}
> final byte[] tmp = new byte[length * 2];
> {code}
> This seems to indicate to me that this buffer will never be able to hold more then 2G worth of data. And likely will hold even less since any length > 1073741824 will cause a integer overflow and turn the new buffer size negative.
> I hope I'm simply missing some critical config setting but it still seems weird that we have a (rather low) upper limit on these buffers. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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