You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "peay (JIRA)" <ji...@apache.org> on 2019/03/04 13:05:00 UTC

[jira] [Updated] (SPARK-27039) toPandas with Arrow swallows maxResultSize errors

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

peay updated SPARK-27039:
-------------------------
    Summary: toPandas with Arrow swallows maxResultSize errors  (was: toPandas with Avro swallows maxResultSize errors)

> toPandas with Arrow swallows maxResultSize errors
> -------------------------------------------------
>
>                 Key: SPARK-27039
>                 URL: https://issues.apache.org/jira/browse/SPARK-27039
>             Project: Spark
>          Issue Type: Bug
>          Components: PySpark
>    Affects Versions: 2.4.0
>            Reporter: peay
>            Priority: Minor
>
> I am running the following simple `toPandas` with {{maxResultSize}} set to 1mb:
> {code:java}
> import pyspark.sql.functions as F
> df = spark.range(1000 * 1000)
> df_pd = df.withColumn("test", F.lit("this is a long string that should make the resulting dataframe too large for maxResult which is 1m")).toPandas()
> {code}
>  
> With {{spark.sql.execution.arrow.enabled}} set to {{true}}, this returns an empty Pandas dataframe without any error:
> {code:python}
> df_pd.info()
> # <class 'pandas.core.frame.DataFrame'>
> # Index: 0 entries
> # Data columns (total 2 columns):
> # id      0 non-null object
> # test    0 non-null object
> # dtypes: object(2)
> # memory usage: 0.0+ bytes
> {code}
> The driver stderr does have an error, and so does the Spark UI:
> {code:java}
> ERROR TaskSetManager: Total size of serialized results of 1 tasks (52.8 MB) is bigger than spark.driver.maxResultSize (1024.0 KB)
> ERROR TaskSetManager: Total size of serialized results of 2 tasks (105.7 MB) is bigger than spark.driver.maxResultSize (1024.0 KB)
> Exception in thread "serve-Arrow" org.apache.spark.SparkException: Job aborted due to stage failure: Total size of serialized results of 1 tasks (52.8 MB) is bigger than spark.driver.maxResultSize (1024.0 KB)
>  at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:2039)
>  at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:2027)
>  at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:2026)
>  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:2026)
>  at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:966)
>  at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:966)
>  at scala.Option.foreach(Option.scala:257)
>  at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:966)
>  at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2260)
>  at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2209)
>  at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2198)
>  at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
>  at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:777)
>  at org.apache.spark.SparkContext.runJob(SparkContext.scala:2061)
>  at org.apache.spark.sql.Dataset$$anonfun$collectAsArrowToPython$1$$anonfun$apply$17.apply(Dataset.scala:3313)
>  at org.apache.spark.sql.Dataset$$anonfun$collectAsArrowToPython$1$$anonfun$apply$17.apply(Dataset.scala:3282)
>  at org.apache.spark.api.python.PythonRDD$$anonfun$6$$anonfun$apply$1.apply$mcV$sp(PythonRDD.scala:435)
>  at org.apache.spark.api.python.PythonRDD$$anonfun$6$$anonfun$apply$1.apply(PythonRDD.scala:435)
>  at org.apache.spark.api.python.PythonRDD$$anonfun$6$$anonfun$apply$1.apply(PythonRDD.scala:435)
>  at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
>  at org.apache.spark.api.python.PythonRDD$$anonfun$6.apply(PythonRDD.scala:436)
>  at org.apache.spark.api.python.PythonRDD$$anonfun$6.apply(PythonRDD.scala:432)
>  at org.apache.spark.api.python.PythonServer$$anon$1.run(PythonRDD.scala:862)
> {code}
> With {{spark.sql.execution.arrow.enabled}} set to {{false}}, the Python call to {{toPandas}} does fail as expected.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org