spark-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Ovidiu-Cristian MARCU <ovidiu-cristian.ma...@inria.fr>
Subject Re: tpcds q1 - java.lang.NegativeArraySizeException
Date Tue, 14 Jun 2016 12:04:50 GMT
I confirm the same exception for other queries as well. I was able to reproduce it many times.
Queries 1, 3 and 5 failed with the same exception. Queries 2 and 4 are running ok.

I am using TPCDSQueryBenchmark and I have used the following settings:

spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true")
spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true”)

 spark.executor.memory              102g
 spark.executor.extraJavaOptions    -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:ObjectAlignmentInBytes=32
 spark.executor.cores               16
 spark.driver.maxResultSize         32g
 spark.default.parallelism          128
 spark.sql.shuffle.partitions       128
 spark.sql.parquet.compression.codec snappy
 spark.sql.optimizer.maxIterations  500
 spark.sql.autoBroadcastJoinThreshold 41943040
 spark.shuffle.file.buffer          64k
 spark.akka.frameSize               128
 spark.shuffle.manager              sort


> On 14 Jun 2016, at 00:12, Sameer Agarwal <sameer@databricks.com> wrote:
> 
> I'm unfortunately not able to reproduce this on master. Does the query always fail deterministically?
> 
> On Mon, Jun 13, 2016 at 12:54 PM, Ovidiu-Cristian MARCU <ovidiu-cristian.marcu@inria.fr
<mailto:ovidiu-cristian.marcu@inria.fr>> wrote:
> Yes, commit ad102af 
> 
>> On 13 Jun 2016, at 21:25, Reynold Xin <rxin@databricks.com <mailto:rxin@databricks.com>>
wrote:
>> 
>> Did you try this on master?
>> 
>> 
>> On Mon, Jun 13, 2016 at 11:26 AM, Ovidiu-Cristian MARCU <ovidiu-cristian.marcu@inria.fr
<mailto:ovidiu-cristian.marcu@inria.fr>> wrote:
>> Hi,
>> 
>> Running the first query of tpcds on a standalone setup (4 nodes, tpcds2 generated
for scale 10 and transformed in parquet under hdfs)  it results in one exception [1].
>> Close to this problem I found this issue https://issues.apache.org/jira/browse/SPARK-12089
<https://issues.apache.org/jira/browse/SPARK-12089> but it seems to be solved.
>> 
>> Running the second query is successful.
>> 
>> OpenJDK 64-Bit Server VM 1.7.0_101-b00 on Linux 3.2.0-4-amd64
>> Intel(R) Xeon(R) CPU E5-2630 v3 @ 2.40GHz
>> TPCDS Snappy:                            Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)
  Relative
>> ------------------------------------------------------------------------------------------------
>> q2                                            4512 / 8142          0.0       61769.4
      1.0X
>> 
>> Best,
>> Ovidiu
>> 
>> [1]
>> WARN TaskSetManager: Lost task 17.0 in stage 80.0 (TID 4469, 172.16.96.70): java.lang.NegativeArraySizeException
>> 	at org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder.grow(BufferHolder.java:61)
>> 	at org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(UnsafeRowWriter.java:214)
>> 	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$doExecute$3$$anon$2.hasNext(WholeStageCodegenExec.scala:386)
>> 	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
>> 	at scala.collection.convert.Wrappers$IteratorWrapper.hasNext(Wrappers.scala:30)
>> 	at org.spark_project.guava.collect.Ordering.leastOf(Ordering.java:628)
>> 	at org.apache.spark.util.collection.Utils$.takeOrdered(Utils.scala:37)
>> 	at org.apache.spark.rdd.RDD$$anonfun$takeOrdered$1$$anonfun$30.apply(RDD.scala:1365)
>> 	at org.apache.spark.rdd.RDD$$anonfun$takeOrdered$1$$anonfun$30.apply(RDD.scala:1362)
>> 	at org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$23.apply(RDD.scala:757)
>> 	at org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$23.apply(RDD.scala:757)
>> 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318)
>> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:282)
>> 	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:1145)
>> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>> 	at java.lang.Thread.run(Thread.java:745)
>> 
>> ERROR TaskSetManager: Task 17 in stage 80.0 failed 4 times; aborting job
>> 
>> Driver stacktrace:
>> 	at org.apache.spark.scheduler.DAGScheduler.org <http://org.apache.spark.scheduler.dagscheduler.org/>$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1435)
>> 	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1423)
>> 	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1422)
>> 	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
>> 	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
>> 	at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1422)
>> 	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:806)
>> 	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:806)
>> 	at scala.Option.foreach(Option.scala:257)
>> 	at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:806)
>> 	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1644)
>> 	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1603)
>> 	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1592)
>> 	at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
>> 	at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:632)
>> 	at org.apache.spark.SparkContext.runJob(SparkContext.scala:1872)
>> 	at org.apache.spark.SparkContext.runJob(SparkContext.scala:1935)
>> 	at org.apache.spark.rdd.RDD$$anonfun$reduce$1.apply(RDD.scala:974)
>> 	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>> 	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
>> 	at org.apache.spark.rdd.RDD.withScope(RDD.scala:357)
>> 	at org.apache.spark.rdd.RDD.reduce(RDD.scala:956)
>> 	at org.apache.spark.rdd.RDD$$anonfun$takeOrdered$1.apply(RDD.scala:1371)
>> 	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>> 	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
>> 	at org.apache.spark.rdd.RDD.withScope(RDD.scala:357)
>> 	at org.apache.spark.rdd.RDD.takeOrdered(RDD.scala:1358)
>> 	at org.apache.spark.sql.execution.TakeOrderedAndProjectExec.executeCollect(limit.scala:128)
>> 	at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2163)
>> 	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:57)
>> 	at org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2489)
>> 	at org.apache.spark.sql.Dataset.org <http://org.apache.spark.sql.dataset.org/>$apache$spark$sql$Dataset$$execute$1(Dataset.scala:2162)
>> 	at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Dataset.scala:2167)
>> 	at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Dataset.scala:2167)
>> 	at org.apache.spark.sql.Dataset.withCallback(Dataset.scala:2502)
>> 	at org.apache.spark.sql.Dataset.org <http://org.apache.spark.sql.dataset.org/>$apache$spark$sql$Dataset$$collect(Dataset.scala:2167)
>> 	at org.apache.spark.sql.Dataset.collect(Dataset.scala:2143)
>> 	at org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmark$$anonfun$tpcdsAll$2$$anonfun$apply$2.apply$mcVI$sp(TPCDSQueryBenchmark.scala:88)
>> 	at org.apache.spark.util.Benchmark$$anonfun$addCase$1.apply(Benchmark.scala:75)
>> 	at org.apache.spark.util.Benchmark$$anonfun$addCase$1.apply(Benchmark.scala:73)
>> 	at org.apache.spark.util.Benchmark.measure(Benchmark.scala:135)
>> 	at org.apache.spark.util.Benchmark$$anonfun$1.apply(Benchmark.scala:104)
>> 	at org.apache.spark.util.Benchmark$$anonfun$1.apply(Benchmark.scala:102)
>> 	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>> 	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>> 	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
>> 	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
>> 	at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
>> 	at scala.collection.AbstractTraversable.map(Traversable.scala:104)
>> 	at org.apache.spark.util.Benchmark.run(Benchmark.scala:102)
>> 	at org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmark$$anonfun$tpcdsAll$2.apply(TPCDSQueryBenchmark.scala:90)
>> 	at org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmark$$anonfun$tpcdsAll$2.apply(TPCDSQueryBenchmark.scala:57)
>> 	at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
>> 	at scala.collection.mutable.WrappedArray.foreach(WrappedArray.scala:35)
>> 	at org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmark$.tpcdsAll(TPCDSQueryBenchmark.scala:57)
>> 	at org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmark$.main(TPCDSQueryBenchmark.scala:135)
>> 	at org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmark.main(TPCDSQueryBenchmark.scala)
>> 	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>> 	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>> 	at java.lang.reflect.Method.invoke(Method.java:606)
>> 	at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:729)
>> 	at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:185)
>> 	at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:210)
>> 	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:124)
>> 	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
>> Caused by: java.lang.NegativeArraySizeException
>> 	at org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder.grow(BufferHolder.java:61)
>> 	at org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter.write(UnsafeRowWriter.java:214)
>> 	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$doExecute$3$$anon$2.hasNext(WholeStageCodegenExec.scala:386)
>> 	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
>> 	at scala.collection.convert.Wrappers$IteratorWrapper.hasNext(Wrappers.scala:30)
>> 	at org.spark_project.guava.collect.Ordering.leastOf(Ordering.java:664)
>> 	at org.apache.spark.util.collection.Utils$.takeOrdered(Utils.scala:37)
>> 	at org.apache.spark.rdd.RDD$$anonfun$takeOrdered$1$$anonfun$30.apply(RDD.scala:1365)
>> 	at org.apache.spark.rdd.RDD$$anonfun$takeOrdered$1$$anonfun$30.apply(RDD.scala:1362)
>> 	at org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$23.apply(RDD.scala:757)
>> 	at org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$23.apply(RDD.scala:757)
>> 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318)
>> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:282)
>> 	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:1145)
>> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>> 	at java.lang.Thread.run(Thread.java:745)
>> 
>> 
> 
> 
> 
> 
> -- 
> Sameer Agarwal
> Software Engineer | Databricks Inc.
> http://cs.berkeley.edu/~sameerag <http://cs.berkeley.edu/~sameerag>

Mime
View raw message