You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Hyukjin Kwon (Jira)" <ji...@apache.org> on 2020/10/14 06:07:00 UTC
[jira] [Commented] (SPARK-33113) [SparkR] gapply works with arrow
disabled, fails with arrow enabled stringsAsFactors=TRUE
[ https://issues.apache.org/jira/browse/SPARK-33113?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17213640#comment-17213640 ]
Hyukjin Kwon commented on SPARK-33113:
--------------------------------------
It works in my local in Spark dev branch:
{code:java}
> df <- as.DataFrame(list("A", "B", "C"), schema="ColumnA")
> udf <- function(key, x) data.frame(out=c("dfs"))
> sparkR.session(master = "local[*]", sparkConfig=list(spark.sql.execution.arrow.sparkr.enabled = "false"))
Java ref type org.apache.spark.sql.SparkSession id 1
> df1 <- gapply(df, c("ColumnA"), udf, "out String")
> collect(df1)
out
1 dfs
2 dfs
3 dfs
> sparkR.session(master = "local[*]", sparkConfig=list(spark.sql.execution.arrow.sparkr.enabled = "true"))
Java ref type org.apache.spark.sql.SparkSession id 1
> df2 <- gapply(df, c("ColumnA"), udf, "out String")
> collect(df2)
out
1 dfs
2 dfs
3 dfs
{code}
> [SparkR] gapply works with arrow disabled, fails with arrow enabled stringsAsFactors=TRUE
> -----------------------------------------------------------------------------------------
>
> Key: SPARK-33113
> URL: https://issues.apache.org/jira/browse/SPARK-33113
> Project: Spark
> Issue Type: Bug
> Components: R
> Affects Versions: 3.0.0, 3.0.1
> Reporter: Jacek Pliszka
> Priority: Major
>
> Running in databricks on Azure
> {code}
> library("arrow")
> library("SparkR")
> df <- as.DataFrame(list("A", "B", "C"), schema="ColumnA")
> udf <- function(key, x) data.frame(out=c("dfs"))
> {code}
>
> This works:
> {code}
> sparkR.session(master = "local[*]", sparkConfig=list(spark.sql.execution.arrow.sparkr.enabled = "false"))
> df1 <- gapply(df, c("ColumnA"), udf, "out String")
> collect(df1)
> {code}
> This fails:
> {code}
> sparkR.session(master = "local[*]", sparkConfig=list(spark.sql.execution.arrow.sparkr.enabled = "true"))
> df2 <- gapply(df, c("ColumnA"), udf, "out String")
> collect(df2)
> {code}
>
> with error
> {code}
> Error in readBin(con, raw(), as.integer(dataLen), endian = "big") : }}Error in readBin(con, raw(), as.integer(dataLen), endian = "big") : invalid 'n' argument
> Error in readBin(con, raw(), as.integer(dataLen), endian = "big") : invalid 'n' argument In addition: Warning messages: 1: Use 'read_ipc_stream' or 'read_feather' instead. 2: Use 'read_ipc_stream' or 'read_feather' instead.
> {code}
>
> Clicking through Failed Stages to Failure Reason:
>
> {code}
> Job aborted due to stage failure: Task 49 in stage 1843.0 failed 4 times, most recent failure: Lost task 49.3 in stage 1843.0 (TID 89810, 10.99.0.5, executor 0): java.lang.UnsupportedOperationException
> at org.apache.spark.sql.vectorized.ArrowColumnVector$ArrowVectorAccessor.getUTF8String(ArrowColumnVector.java:233)
> at org.apache.spark.sql.vectorized.ArrowColumnVector.getUTF8String(ArrowColumnVector.java:109)
> at org.apache.spark.sql.vectorized.ColumnarBatchRow.getUTF8String(ColumnarBatch.java:220)
> at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
> at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
> at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
> at org.apache.spark.sql.execution.arrow.ArrowConverters$$anon$1.$anonfun$next$1(ArrowConverters.scala:131)
> at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
> at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1559)
> at org.apache.spark.sql.execution.arrow.ArrowConverters$$anon$1.next(ArrowConverters.scala:140)
> at org.apache.spark.sql.execution.arrow.ArrowConverters$$anon$1.next(ArrowConverters.scala:115)
> at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
> at scala.collection.Iterator.foreach(Iterator.scala:941)
> at scala.collection.Iterator.foreach$(Iterator.scala:941)
> at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
> at scala.collection.generic.Growable.$plus$plus$eq(Growable.scala:62)
> at scala.collection.generic.Growable.$plus$plus$eq$(Growable.scala:53)
> at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:105)
> at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:49)
> at scala.collection.TraversableOnce.to(TraversableOnce.scala:315)
> at scala.collection.TraversableOnce.to$(TraversableOnce.scala:313)
> at scala.collection.AbstractIterator.to(Iterator.scala:1429)
> at scala.collection.TraversableOnce.toBuffer(TraversableOnce.scala:307)
> at scala.collection.TraversableOnce.toBuffer$(TraversableOnce.scala:307)
> at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1429)
> at scala.collection.TraversableOnce.toArray(TraversableOnce.scala:294)
> at scala.collection.TraversableOnce.toArray$(TraversableOnce.scala:288)
> at scala.collection.AbstractIterator.toArray(Iterator.scala:1429)
> at org.apache.spark.sql.Dataset.$anonfun$collectAsArrowToR$3(Dataset.scala:3589)
> at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
> at org.apache.spark.scheduler.Task.doRunTask(Task.scala:144)
> at org.apache.spark.scheduler.Task.run(Task.scala:117)
> at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$9(Executor.scala:639)
> at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1559)
> at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:642)
> at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> Driver stacktrace:
> {code}
>
>
>
>
>
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org