You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "ding (JIRA)" <ji...@apache.org> on 2016/06/21 07:55:58 UTC

[jira] [Commented] (SPARK-16071) Not sufficient array size checks to avoid integer overflows in Tungsten

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

ding commented on SPARK-16071:
------------------------------

The exception raised in different location as one happened in encoder(n = 3e8) and the other happened in executing logical plan(n = 2e8) when createDataset. And both of exceptions are thrown from grow function in BufferHolder. I think we can add array size check here. I will send a PR later.

> Not sufficient array size checks to avoid integer overflows in Tungsten
> -----------------------------------------------------------------------
>
>                 Key: SPARK-16071
>                 URL: https://issues.apache.org/jira/browse/SPARK-16071
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.0.0
>            Reporter: Xiangrui Meng
>            Priority: Critical
>
> Several bugs have been found caused by integer overflows in Tungsten. This JIRA is for taking a final pass before 2.0 release to reduce potential bugs and issues. We should do at least the following:
> * Raise exception early instead of later throwing NegativeArraySize (which is slow and might cause silent errors)
> * Document clearly the largest array size we support in DataFrames.
> To reproduce one of the issues:
> {code}
> val n = 1e8.toInt // try 2e8, 3e8
> sc.parallelize(0 until 1, 1).map(i => new Array[Int](n)).toDS.map(_.size).show()
> {code}
> Result:
> * n=1e8: correct but slow (see SPARK-16043)
> * n=2e8: NegativeArraySize exception
> {code:none}
> java.lang.NegativeArraySizeException
> 	at org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder.grow(BufferHolder.java:61)
> 	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
> 	at org.apache.spark.sql.execution.RDDScanExec$$anonfun$doExecute$1$$anonfun$apply$3.apply(ExistingRDD.scala:123)
> 	at org.apache.spark.sql.execution.RDDScanExec$$anonfun$doExecute$1$$anonfun$apply$3.apply(ExistingRDD.scala:121)
> 	at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
> 	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$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
> 	at org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:246)
> 	at org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:240)
> 	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:780)
> 	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:780)
> 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
> 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
> 	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:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> {code}
> * n=3e8: NegativeArraySize exception but raised at a different location
> {code:none}
> java.lang.RuntimeException: Error while encoding: java.lang.NegativeArraySizeException
> newInstance(class org.apache.spark.sql.catalyst.util.GenericArrayData) AS value#108
> +- newInstance(class org.apache.spark.sql.catalyst.util.GenericArrayData)
>    +- input[0, [I, true]
> 	at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:257)
> 	at org.apache.spark.sql.SparkSession$$anonfun$4.apply(SparkSession.scala:430)
> 	at org.apache.spark.sql.SparkSession$$anonfun$4.apply(SparkSession.scala:430)
> 	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.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$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
> 	at org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:246)
> 	at org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:240)
> 	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:780)
> 	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:780)
> 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
> 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
> 	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:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> {code}



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