Description
I created a DataFrame from the 1000 genomes data set using csv data source. I register it as a table and tried to cache it. I get the following error:
val vcfData = sqlContext.read.format("csv").options(Map( "comment" -> "#", "header" -> "false", "delimiter" -> "\t" )).load("/1000genomes/") val colNames = sc.textFile("/hossein/1000genomes/").take(100).filter(_.startsWith("#CHROM")).head.split("\t") val data = vcfData.toDF(colNames: _*).registerTempTable("genomeTable) %sql cache table genomeTable org.apache.spark.SparkException: Job aborted due to stage failure: Total size of serialized results of 2086 tasks (4.0 GB) is bigger than spark.driver.maxResultSize (4.0 GB) at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1457) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1445) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1444) 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:1444) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:809) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:809) at scala.Option.foreach(Option.scala:236) at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:809) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1666) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1625) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1614) at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48) at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:630) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1765) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1778) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1791) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1805) at org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:881) 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.collect(RDD.scala:880) at org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:276) at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:1979) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:53) at org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2242) at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$execute$1(Dataset.scala:1978) at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$collect(Dataset.scala:1985) at org.apache.spark.sql.Dataset$$anonfun$count$1.apply(Dataset.scala:1995) at org.apache.spark.sql.Dataset$$anonfun$count$1.apply(Dataset.scala:1994) at org.apache.spark.sql.Dataset.withCallback(Dataset.scala:2255) at org.apache.spark.sql.Dataset.count(Dataset.scala:1994) at org.apache.spark.sql.execution.command.CacheTableCommand.run(commands.scala:270) at org.apache.spark.sql.execution.command.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:61) at org.apache.spark.sql.execution.command.ExecutedCommand.sideEffectResult(commands.scala:59) at org.apache.spark.sql.execution.command.ExecutedCommand.doExecute(commands.scala:73) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:118) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:118) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:137) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:134) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:117) at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:60) at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:60) at org.apache.spark.sql.Dataset.<init>(Dataset.scala:179) at org.apache.spark.sql.Dataset.<init>(Dataset.scala:164) at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:59) at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:748)
Attachments
Issue Links
- is duplicated by
-
SPARK-12837 Spark driver requires large memory space for serialized results even there are no data collected to the driver
- Resolved