You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Michael Armbrust (JIRA)" <ji...@apache.org> on 2015/10/22 23:21:27 UTC

[jira] [Reopened] (SPARK-11229) NPE in JoinedRow.isNullAt when spark.shuffle.memoryFraction=0

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

Michael Armbrust reopened SPARK-11229:
--------------------------------------

> NPE in JoinedRow.isNullAt when spark.shuffle.memoryFraction=0
> -------------------------------------------------------------
>
>                 Key: SPARK-11229
>                 URL: https://issues.apache.org/jira/browse/SPARK-11229
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 1.5.1
>         Environment: 14.04.1-Ubuntu SMP x86_64 GNU/Linux
>            Reporter: Romi Kuntsman
>             Fix For: 1.6.0
>
>
> Steps to reproduce:
> 1. set spark.shuffle.memoryFraction=0
> 2. load dataframe from parquet file
> 3. see it's read correctly by calling dataframe.show()
> 4. call dataframe.count()
> Expected behaviour:
> get count of rows in dataframe
> OR, if memoryFraction=0 is an invalid setting, get notified about it
> Actual behaviour:
> CatalystReadSupport doesn't read the schema (even thought there is one) and then there's a NullPointerException.
> Driver stacktrace:
> 	at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1283)
> 	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1271)
> 	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1270)
> 	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> 	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
> 	at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1270)
> 	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:697)
> 	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:697)
> 	at scala.Option.foreach(Option.scala:236)
> 	at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:697)
> 	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1496)
> 	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1458)
> 	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1447)
> 	at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
> 	at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:567)
> 	at org.apache.spark.SparkContext.runJob(SparkContext.scala:1822)
> 	at org.apache.spark.SparkContext.runJob(SparkContext.scala:1835)
> 	at org.apache.spark.SparkContext.runJob(SparkContext.scala:1848)
> 	at org.apache.spark.SparkContext.runJob(SparkContext.scala:1919)
> 	at org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:905)
> 	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:147)
> 	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:108)
> 	at org.apache.spark.rdd.RDD.withScope(RDD.scala:306)
> 	at org.apache.spark.rdd.RDD.collect(RDD.scala:904)
> 	at org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:177)
> 	at org.apache.spark.sql.DataFrame$$anonfun$collect$1.apply(DataFrame.scala:1385)
> 	at org.apache.spark.sql.DataFrame$$anonfun$collect$1.apply(DataFrame.scala:1385)
> 	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:56)
> 	at org.apache.spark.sql.DataFrame.withNewExecutionId(DataFrame.scala:1903)
> 	at org.apache.spark.sql.DataFrame.collect(DataFrame.scala:1384)
> 	at org.apache.spark.sql.DataFrame.count(DataFrame.scala:1402)
> 	... 14 more
> Caused by: java.lang.NullPointerException
> 	at org.apache.spark.sql.catalyst.expressions.JoinedRow.isNullAt(JoinedRow.scala:70)
> 	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificMutableProjection.apply(Unknown Source)
> 	at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator$$anonfun$generateProcessRow$1.apply(TungstenAggregationIterator.scala:194)
> 	at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator$$anonfun$generateProcessRow$1.apply(TungstenAggregationIterator.scala:192)
> 	at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.processInputs(TungstenAggregationIterator.scala:368)
> 	at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.start(TungstenAggregationIterator.scala:622)
> 	at org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1.org$apache$spark$sql$execution$aggregate$TungstenAggregate$$anonfun$$executePartition$1(TungstenAggregate.scala:110)
> 	at org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:119)
> 	at org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:119)
> 	at org.apache.spark.rdd.MapPartitionsWithPreparationRDD.compute(MapPartitionsWithPreparationRDD.scala:64)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
> 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
> 	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:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> Worker log:
> 15/10/21 12:06:36 INFO CoarseGrainedExecutorBackend: Got assigned task 2
> 15/10/21 12:06:36 INFO Executor: Running task 0.0 in stage 1.0 (TID 2)
> 15/10/21 12:06:36 INFO TorrentBroadcast: Started reading broadcast variable 2
> 15/10/21 12:06:36 INFO MemoryStore: ensureFreeSpace(5275) called with curMem=69383, maxMem=2793500835
> 15/10/21 12:06:36 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 5.2 KB, free 2.6 GB)
> 15/10/21 12:06:36 INFO TorrentBroadcast: Reading broadcast variable 2 took 9 ms
> 15/10/21 12:06:36 INFO MemoryStore: ensureFreeSpace(10432) called with curMem=74658, maxMem=2793500835
> 15/10/21 12:06:36 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 10.2 KB, free 2.6 GB)
> 15/10/21 12:06:36 INFO GenerateMutableProjection: Code generated in 70.404364 ms
> 15/10/21 12:06:36 INFO GenerateUnsafeProjection: Code generated in 6.235261 ms
> 15/10/21 12:06:36 INFO GenerateMutableProjection: Code generated in 10.861097 ms
> 15/10/21 12:06:36 INFO GenerateUnsafeRowJoiner: Code generated in 5.404177 ms
> 15/10/21 12:06:36 INFO GenerateUnsafeProjection: Code generated in 4.892669 ms
> 15/10/21 12:06:36 INFO ParquetRelation$$anonfun$buildScan$1$$anon$1: Input split: ParquetInputSplit{part: file:/home/user/parquet/part-r-00001.gz.parquet start: 0 end: 178913 length: 178913 hosts: []}
> 15/10/21 12:06:36 INFO TorrentBroadcast: Started reading broadcast variable 1
> 15/10/21 12:06:36 INFO MemoryStore: ensureFreeSpace(15856) called with curMem=85090, maxMem=2793500835
> 15/10/21 12:06:36 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 15.5 KB, free 2.6 GB)
> 15/10/21 12:06:36 INFO TorrentBroadcast: Reading broadcast variable 1 took 9 ms
> 15/10/21 12:06:36 INFO MemoryStore: ensureFreeSpace(196360) called with curMem=100946, maxMem=2793500835
> 15/10/21 12:06:36 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 191.8 KB, free 2.6 GB)
> 15/10/21 12:06:36 INFO deprecation: mapred.min.split.size is deprecated. Instead, use mapreduce.input.fileinputformat.split.minsize
> 15/10/21 12:06:36 WARN ParquetRecordReader: Can not initialize counter due to context is not a instance of TaskInputOutputContext, but is org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
> 15/10/21 12:06:36 INFO CatalystReadSupport: Going to read the following fields from the Parquet file:
> Parquet form:
> message root {
> }
> Catalyst form:
> StructType()
>        
> 15/10/21 12:06:36 INFO InternalParquetRecordReader: RecordReader initialized will read a total of 36833 records.
> 15/10/21 12:06:36 INFO InternalParquetRecordReader: at row 0. reading next block
> 15/10/21 12:06:36 INFO InternalParquetRecordReader: block read in memory in 2 ms. row count = 36833
> 15/10/21 12:06:36 ERROR Executor: Exception in task 0.0 in stage 1.0 (TID 2)
> java.lang.NullPointerException
> 	at org.apache.spark.sql.catalyst.expressions.JoinedRow.isNullAt(JoinedRow.scala:70)
> 	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificMutableProjection.apply(Unknown Source)
> 	at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator$$anonfun$generateProcessRow$1.apply(TungstenAggregationIterator.scala:194)
> 	at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator$$anonfun$generateProcessRow$1.apply(TungstenAggregationIterator.scala:192)
> 	at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.processInputs(TungstenAggregationIterator.scala:368)
> 	at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.start(TungstenAggregationIterator.scala:622)
> 	at org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1.org$apache$spark$sql$execution$aggregate$TungstenAggregate$$anonfun$$executePartition$1(TungstenAggregate.scala:110)
> 	at org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:119)
> 	at org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:119)
> 	at org.apache.spark.rdd.MapPartitionsWithPreparationRDD.compute(MapPartitionsWithPreparationRDD.scala:64)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
> 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
> 	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:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)



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