[ https://issues.apache.org/jira/browse/SPARK-45657?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17779281#comment-17779281 ]
John Zhuge edited comment on SPARK-45657 at 10/25/23 12:38 AM: --------------------------------------------------------------- Root cause: # SQL UNION of 2 sides with different data types produce a Project of Project on 1 side to cast the type. When this is cached, the Project of Project is preserved. {noformat} Distinct +- Union false, false :- Project [cast(id#153 as string) AS id#155] : +- Project [1 AS id#153] : +- OneRowRelation +- Project [s2 AS id#154] +- OneRowRelation{noformat} # Dataset.union applies `CombineUnions` which applies to all unions in the tree. CombineUnions collapses the 2 Projects into 1, thus Dataset.union of the above plan with any plan will not be find a matching cached plan. {code:java} object CombineUnions extends Rule[LogicalPlan] { ... private def flattenUnion(union: Union, flattenDistinct: Boolean): ... case p1 @ Project(_, p2: Project) if canCollapseExpressions(p1.projectList, p2.projectList, alwaysInline = false) && !p1.projectList.exists(SubqueryExpression.hasCorrelatedSubquery) && !p2.projectList.exists(SubqueryExpression.hasCorrelatedSubquery) => val newProjectList = buildCleanedProjectList(p1.projectList, p2.projectList) stack.pushAll(Seq(p2.copy(projectList = newProjectList))){code} was (Author: jzhuge): Root cause: # SQL UNION of 2 sides with different data types produce a Project of Project on 1 side to cast the type. When this is cached, the Project of Project is preserved. {noformat} Distinct +- Union false, false :- Project [cast(id#153 as string) AS id#155] : +- Project [1 AS id#153] : +- OneRowRelation +- Project [s2 AS id#154] +- OneRowRelation{noformat} # Dataset.union applies `CombineUnions` which applies to all unions in the tree. CombineUnions collapses the 2 Projects into 1, thus Dataset.union of the above plan with any plan will not be find a matching cached plan. > Caching SQL UNION of different column data types does not work inside > Dataset.union > ----------------------------------------------------------------------------------- > > Key: SPARK-45657 > URL: https://issues.apache.org/jira/browse/SPARK-45657 > Project: Spark > Issue Type: Bug > Components: SQL > Affects Versions: 3.4.1 > Reporter: John Zhuge > Priority: Major > > > Cache SQL UNION of 2 sides with different column data types > {code:java} > scala> spark.sql("select 1 id union select 's2' id").cache() {code} > Dataset.union does not leverage the cache > {code:java} > scala> spark.sql("select 1 id union select 's2' id").union(spark.sql("select > 's3'")).queryExecution.optimizedPlan > res15: org.apache.spark.sql.catalyst.plans.logical.LogicalPlan = > Union false, false > :- Aggregate [id#109], [id#109] > : +- Union false, false > : :- Project [1 AS id#109] > : : +- OneRowRelation > : +- Project [s2 AS id#108] > : +- OneRowRelation > +- Project [s3 AS s3#111] > +- OneRowRelation {code} > SQL UNION of the cached SQL UNION does use the cache! Please note > `InMemoryRelation` used. > {code:java} > scala> spark.sql("(select 1 id union select 's2' id) union select > 's3'").queryExecution.optimizedPlan > res16: org.apache.spark.sql.catalyst.plans.logical.LogicalPlan = > Aggregate [id#117], [id#117] > +- Union false, false > :- InMemoryRelation [id#117], StorageLevel(disk, memory, deserialized, 1 > replicas) > : +- *(4) HashAggregate(keys=[id#100], functions=[], output=[id#100]) > : +- Exchange hashpartitioning(id#100, 500), ENSURE_REQUIREMENTS, > [plan_id=241] > : +- *(3) HashAggregate(keys=[id#100], functions=[], > output=[id#100]) > : +- Union > : :- *(1) Project [1 AS id#100] > : : +- *(1) Scan OneRowRelation[] > : +- *(2) Project [s2 AS id#99] > : +- *(2) Scan OneRowRelation[] > +- Project [s3 AS s3#116] > +- OneRowRelation {code} > -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org