diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 4b4722b0b211..b1ffdca09146 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -284,7 +284,7 @@ object QueryPlan extends PredicateHelper { if (ordinal == -1) { ar } else { - ar.withExprId(ExprId(ordinal)) + ar.withExprId(ExprId(ordinal)).canonicalized } }.canonicalized.asInstanceOf[T] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala index aaf51b5b9011..d088e24e53bf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala @@ -18,8 +18,11 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{DataFrame, QueryTest} +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.IntegerType /** * Tests for the sameResult function for [[SparkPlan]]s. @@ -58,4 +61,16 @@ class SameResultSuite extends QueryTest with SharedSQLContext { val df4 = spark.range(10).agg(sumDistinct($"id")) assert(df3.queryExecution.executedPlan.sameResult(df4.queryExecution.executedPlan)) } + + test("Canonicalized result is case-insensitive") { + val a = AttributeReference("A", IntegerType)() + val b = AttributeReference("B", IntegerType)() + val planUppercase = Project(Seq(a), LocalRelation(a, b)) + + val c = AttributeReference("a", IntegerType)() + val d = AttributeReference("b", IntegerType)() + val planLowercase = Project(Seq(c), LocalRelation(c, d)) + + assert(planUppercase.sameResult(planLowercase)) + } }