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

[jira] [Created] (SPARK-12089) java.lang.NegativeArraySizeException when growing BufferHolder

Erik Selin created SPARK-12089:
----------------------------------

             Summary: 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


When running a rather 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 code it looks like this is due to a doubling of a target length that in my case makes the new buffer length flip to negative due to what i assume is just too much data?

The offending line {{BufferHolder.java:36}} is the following in the version I'm running:

{code}
final byte[] tmp = new byte[length * 2];
{code}

I'm still digging down to try to pin point what is actually responsible for managing how big this should be grown. I figure we cannot simply add some check here to keep it from going negative since arguably we do need the buffer to grow this big?



--
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