You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Herman van Hovell (JIRA)" <ji...@apache.org> on 2018/06/11 09:04:00 UTC

[jira] [Assigned] (SPARK-24500) UnsupportedOperationException when trying to execute Union plan with Stream of children

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

Herman van Hovell reassigned SPARK-24500:
-----------------------------------------

    Assignee: Herman van Hovell

> UnsupportedOperationException when trying to execute Union plan with Stream of children
> ---------------------------------------------------------------------------------------
>
>                 Key: SPARK-24500
>                 URL: https://issues.apache.org/jira/browse/SPARK-24500
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.3.0
>            Reporter: Bogdan Raducanu
>            Assignee: Herman van Hovell
>            Priority: Major
>
> To reproduce:
> {code}
> import org.apache.spark.sql.catalyst.plans.logical._
> def range(i: Int) = Range(1, i, 1, 1)
> val union = Union(Stream(range(3), range(5), range(7)))
> spark.sessionState.planner.plan(union).next().execute()
> {code}
> produces
> {code}
> java.lang.UnsupportedOperationException
>   at org.apache.spark.sql.execution.PlanLater.doExecute(SparkStrategies.scala:55)
>   at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
>   at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
>   at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
>   at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>   at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
> {code}
> The SparkPlan looks like this:
> {code}
> :- Range (1, 3, step=1, splits=1)
> :- PlanLater Range (1, 5, step=1, splits=Some(1))
> +- PlanLater Range (1, 7, step=1, splits=Some(1))
> {code}
> So not all of it was planned (some PlanLater still in there).
> This appears to be a longstanding issue.
> I traced it to the use of var in TreeNode.
> For example in mapChildren:
> {code}
>         case args: Traversable[_] => args.map {
>           case arg: TreeNode[_] if containsChild(arg) =>
>             val newChild = f(arg.asInstanceOf[BaseType])
>             if (!(newChild fastEquals arg)) {
>               changed = true
> {code}
> If args is a Stream then changed will never be set here, ultimately causing the method to return the original plan.



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