diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index e8ec72b7c802..c6dcde59e246 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -141,7 +141,6 @@ package object dsl { def desc: SortOrder = SortOrder(expr, Descending) def desc_nullsFirst: SortOrder = SortOrder(expr, Descending, NullsFirst, Seq.empty) def as(alias: String): NamedExpression = Alias(expr, alias)() - def as(alias: Symbol): NamedExpression = Alias(expr, alias.name)() } trait ExpressionConversions { @@ -166,9 +165,6 @@ package object dsl { implicit def instantToLiteral(i: Instant): Literal = Literal(i) implicit def binaryToLiteral(a: Array[Byte]): Literal = Literal(a) - implicit def symbolToUnresolvedAttribute(s: Symbol): analysis.UnresolvedAttribute = - analysis.UnresolvedAttribute(s.name) - /** Converts $"col name" into an [[analysis.UnresolvedAttribute]]. */ implicit class StringToAttributeConversionHelper(val sc: StringContext) { // Note that if we make ExpressionConversions an object rather than a trait, we can @@ -244,7 +240,6 @@ package object dsl { def windowExpr(windowFunc: Expression, windowSpec: WindowSpecDefinition): WindowExpression = WindowExpression(windowFunc, windowSpec) - implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s: String = sym.name } // TODO more implicit class for literal? implicit class DslString(val s: String) extends ImplicitOperators { override def expr: Expression = Literal(s) @@ -308,10 +303,10 @@ package object dsl { AttributeReference(s, arrayType)() /** Creates a new AttributeReference of type map */ - def map(keyType: DataType, valueType: DataType): AttributeReference = - map(MapType(keyType, valueType)) + def mapAttr(keyType: DataType, valueType: DataType): AttributeReference = + mapAttr(MapType(keyType, valueType)) - def map(mapType: MapType): AttributeReference = + def mapAttr(mapType: MapType): AttributeReference = AttributeReference(s, mapType, nullable = true)() /** Creates a new AttributeReference of type struct */ @@ -414,8 +409,6 @@ package object dsl { orderSpec: Seq[SortOrder]): LogicalPlan = Window(windowExpressions, partitionSpec, orderSpec, logicalPlan) - def subquery(alias: Symbol): LogicalPlan = SubqueryAlias(alias.name, logicalPlan) - def except(otherPlan: LogicalPlan, isAll: Boolean): LogicalPlan = Except(logicalPlan, otherPlan, isAll) @@ -443,6 +436,7 @@ package object dsl { InsertIntoStatement(table, partition, Nil, logicalPlan, overwrite, ifPartitionNotExists) def as(alias: String): LogicalPlan = SubqueryAlias(alias, logicalPlan) + def subquery(alias: String): LogicalPlan = as(alias) def coalesce(num: Integer): LogicalPlan = Repartition(num, shuffle = false, logicalPlan) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 01d223d18b32..5b33264ccd0c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -624,20 +624,20 @@ class AnalysisErrorSuite extends AnalysisTest { } test("Join can work on binary types but can't work on map types") { - val left = LocalRelation(Symbol("a").binary, Symbol("b").map(StringType, StringType)) - val right = LocalRelation(Symbol("c").binary, Symbol("d").map(StringType, StringType)) + val left = LocalRelation("a".attr.binary, "b".attr.mapAttr(StringType, StringType)) + val right = LocalRelation("c".attr.binary, "d".attr.mapAttr(StringType, StringType)) val plan1 = left.join( right, joinType = Cross, - condition = Some(Symbol("a") === Symbol("c"))) + condition = Some("a".attr === "c".attr)) assertAnalysisSuccess(plan1) val plan2 = left.join( right, joinType = Cross, - condition = Some(Symbol("b") === Symbol("d"))) + condition = Some("b".attr === "d".attr)) assertAnalysisError(plan2, "EqualTo does not support ordering on type map" :: Nil) } @@ -705,7 +705,7 @@ class AnalysisErrorSuite extends AnalysisTest { test("Error on filter condition containing aggregate expressions") { val a = AttributeReference("a", IntegerType)() val b = AttributeReference("b", IntegerType)() - val plan = Filter(Symbol("a") === UnresolvedFunction("max", Seq(b), true), LocalRelation(a, b)) + val plan = Filter("a".attr === UnresolvedFunction("max", Seq(b), true), LocalRelation(a, b)) assertAnalysisError(plan, "Aggregate/Window/Generate expressions are not valid in where clause of the query" :: Nil) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index a3c26ecdaba2..25b8a7079eee 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -52,7 +52,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { test("fail for unresolved plan") { intercept[AnalysisException] { // `testRelation` does not have column `b`. - testRelation.select('b).analyze + testRelation.select("b".attr).analyze } } @@ -285,7 +285,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { CreateNamedStruct(Seq( Literal(att1.name), att1, Literal("a_plus_1"), (att1 + 1))), - Symbol("col").struct(prevPlan.output(0).dataType.asInstanceOf[StructType]).notNull + "col".attr.struct(prevPlan.output(0).dataType.asInstanceOf[StructType]).notNull )).as("arr") ) @@ -426,15 +426,15 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("SPARK-12102: Ignore nullability when comparing two sides of case") { - val relation = LocalRelation(Symbol("a").struct(Symbol("x").int), - Symbol("b").struct(Symbol("x").int.withNullability(false))) + val relation = LocalRelation("a".attr.struct("x".attr.int), + "b".attr.struct("x".attr.int.withNullability(false))) val plan = relation.select( - CaseWhen(Seq((Literal(true), Symbol("a").attr)), Symbol("b")).as("val")) + CaseWhen(Seq((Literal(true), "a".attr)), "b".attr).as("val")) assertAnalysisSuccess(plan) } test("Keep attribute qualifiers after dedup") { - val input = LocalRelation(Symbol("key").int, Symbol("value").string) + val input = LocalRelation("key".attr.int, "value".attr.string) val query = Project(Seq($"x.key", $"y.key"), @@ -561,13 +561,13 @@ class AnalysisSuite extends AnalysisTest with Matchers { test("SPARK-20963 Support aliases for join relations in FROM clause") { def joinRelationWithAliases(outputNames: Seq[String]): LogicalPlan = { - val src1 = LocalRelation(Symbol("id").int, Symbol("v1").string).as("s1") - val src2 = LocalRelation(Symbol("id").int, Symbol("v2").string).as("s2") + val src1 = LocalRelation("id".attr.int, "v1".attr.string).as("s1") + val src2 = LocalRelation("id".attr.int, "v2".attr.string).as("s2") UnresolvedSubqueryColumnAliases( outputNames, SubqueryAlias( "dst", - src1.join(src2, Inner, Option(Symbol("s1.id") === Symbol("s2.id")))) + src1.join(src2, Inner, Option("s1.id".attr === "s2.id".attr))) ).select(star()) } assertAnalysisSuccess(joinRelationWithAliases("col1" :: "col2" :: "col3" :: "col4" :: Nil)) @@ -591,12 +591,12 @@ class AnalysisSuite extends AnalysisTest with Matchers { checkPartitioning[HashPartitioning](numPartitions = 10, exprs = Literal(20)) checkPartitioning[HashPartitioning](numPartitions = 10, - exprs = Symbol("a").attr, Symbol("b").attr) + exprs = "a".attr, "b".attr) checkPartitioning[RangePartitioning](numPartitions = 10, exprs = SortOrder(Literal(10), Ascending)) checkPartitioning[RangePartitioning](numPartitions = 10, - exprs = SortOrder(Symbol("a").attr, Ascending), SortOrder(Symbol("b").attr, Descending)) + exprs = SortOrder("a".attr, Ascending), SortOrder("b".attr, Descending)) checkPartitioning[RoundRobinPartitioning](numPartitions = 10, exprs = Seq.empty: _*) @@ -608,7 +608,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { } intercept[IllegalArgumentException] { checkPartitioning(numPartitions = 10, exprs = - SortOrder(Symbol("a").attr, Ascending), Symbol("b").attr) + SortOrder("a".attr, Ascending), "b".attr) } } @@ -779,7 +779,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { "Multiple definitions of observed metrics" :: "evt1" :: Nil) // Different children, same metrics - fail - val b = Symbol("b").string + val b = "b".attr.string val tblB = LocalRelation(b) assertAnalysisError(Union( CollectMetrics("evt1", count :: Nil, testRelation) :: diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DSLHintSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DSLHintSuite.scala index c316e0406b1b..068959564573 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DSLHintSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DSLHintSuite.scala @@ -22,9 +22,9 @@ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.plans.logical._ class DSLHintSuite extends AnalysisTest { - lazy val a = Symbol("a").int - lazy val b = Symbol("b").string - lazy val c = Symbol("c").string + lazy val a = "a".attr.int + lazy val b = "b".attr.string + lazy val c = "c".attr.string lazy val r1 = LocalRelation(a, b, c) test("various hint parameters") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index 46634c93148b..ff5b139f5aa9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -29,12 +29,12 @@ import org.apache.spark.sql.types._ class ExpressionTypeCheckingSuite extends SparkFunSuite { val testRelation = LocalRelation( - Symbol("intField").int, - Symbol("stringField").string, - Symbol("booleanField").boolean, - Symbol("decimalField").decimal(8, 0), - Symbol("arrayField").array(StringType), - Symbol("mapField").map(StringType, LongType)) + "intField".attr.int, + "stringField".attr.string, + "booleanField".attr.boolean, + "decimalField".attr.decimal(8, 0), + "arrayField".attr.array(StringType), + "mapField".attr.mapAttr(StringType, LongType)) def assertError(expr: Expression, errorMessage: String): Unit = { val e = intercept[AnalysisException] { @@ -56,92 +56,92 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { } test("check types for unary arithmetic") { - assertError(BitwiseNot(Symbol("stringField")), "requires integral type") + assertError(BitwiseNot("stringField".attr), "requires integral type") } test("check types for binary arithmetic") { // We will cast String to Double for binary arithmetic - assertSuccess(Add(Symbol("intField"), Symbol("stringField"))) - assertSuccess(Subtract(Symbol("intField"), Symbol("stringField"))) - assertSuccess(Multiply(Symbol("intField"), Symbol("stringField"))) - assertSuccess(Divide(Symbol("intField"), Symbol("stringField"))) - assertSuccess(Remainder(Symbol("intField"), Symbol("stringField"))) - // checkAnalysis(BitwiseAnd(Symbol("intField"), Symbol("stringField"))) - - assertErrorForDifferingTypes(Add(Symbol("intField"), Symbol("booleanField"))) - assertErrorForDifferingTypes(Subtract(Symbol("intField"), Symbol("booleanField"))) - assertErrorForDifferingTypes(Multiply(Symbol("intField"), Symbol("booleanField"))) - assertErrorForDifferingTypes(Divide(Symbol("intField"), Symbol("booleanField"))) - assertErrorForDifferingTypes(Remainder(Symbol("intField"), Symbol("booleanField"))) - assertErrorForDifferingTypes(BitwiseAnd(Symbol("intField"), Symbol("booleanField"))) - assertErrorForDifferingTypes(BitwiseOr(Symbol("intField"), Symbol("booleanField"))) - assertErrorForDifferingTypes(BitwiseXor(Symbol("intField"), Symbol("booleanField"))) - - assertError(Add(Symbol("booleanField"), Symbol("booleanField")), + assertSuccess(Add("intField".attr, "stringField".attr)) + assertSuccess(Subtract("intField".attr, "stringField".attr)) + assertSuccess(Multiply("intField".attr, "stringField".attr)) + assertSuccess(Divide("intField".attr, "stringField".attr)) + assertSuccess(Remainder("intField".attr, "stringField".attr)) + // checkAnalysis(BitwiseAnd("intField".attr, "stringField".attr) + + assertErrorForDifferingTypes(Add("intField".attr, "booleanField".attr)) + assertErrorForDifferingTypes(Subtract("intField".attr, "booleanField".attr)) + assertErrorForDifferingTypes(Multiply("intField".attr, "booleanField".attr)) + assertErrorForDifferingTypes(Divide("intField".attr, "booleanField".attr)) + assertErrorForDifferingTypes(Remainder("intField".attr, "booleanField".attr)) + assertErrorForDifferingTypes(BitwiseAnd("intField".attr, "booleanField".attr)) + assertErrorForDifferingTypes(BitwiseOr("intField".attr, "booleanField".attr)) + assertErrorForDifferingTypes(BitwiseXor("intField".attr, "booleanField".attr)) + + assertError(Add("booleanField".attr, "booleanField".attr), "requires (numeric or interval) type") - assertError(Subtract(Symbol("booleanField"), Symbol("booleanField")), + assertError(Subtract("booleanField".attr, "booleanField".attr), "requires (numeric or interval) type") - assertError(Multiply(Symbol("booleanField"), Symbol("booleanField")), "requires numeric type") - assertError(Divide(Symbol("booleanField"), Symbol("booleanField")), + assertError(Multiply("booleanField".attr, "booleanField".attr), "requires numeric type") + assertError(Divide("booleanField".attr, "booleanField".attr), "requires (double or decimal) type") - assertError(Remainder(Symbol("booleanField"), Symbol("booleanField")), "requires numeric type") + assertError(Remainder("booleanField".attr, "booleanField".attr), "requires numeric type") - assertError(BitwiseAnd(Symbol("booleanField"), Symbol("booleanField")), + assertError(BitwiseAnd("booleanField".attr, "booleanField".attr), "requires integral type") - assertError(BitwiseOr(Symbol("booleanField"), Symbol("booleanField")), "requires integral type") - assertError(BitwiseXor(Symbol("booleanField"), Symbol("booleanField")), + assertError(BitwiseOr("booleanField".attr, "booleanField".attr), "requires integral type") + assertError(BitwiseXor("booleanField".attr, "booleanField".attr), "requires integral type") } test("check types for predicates") { // We will cast String to Double for binary comparison - assertSuccess(EqualTo(Symbol("intField"), Symbol("stringField"))) - assertSuccess(EqualNullSafe(Symbol("intField"), Symbol("stringField"))) - assertSuccess(LessThan(Symbol("intField"), Symbol("stringField"))) - assertSuccess(LessThanOrEqual(Symbol("intField"), Symbol("stringField"))) - assertSuccess(GreaterThan(Symbol("intField"), Symbol("stringField"))) - assertSuccess(GreaterThanOrEqual(Symbol("intField"), Symbol("stringField"))) + assertSuccess(EqualTo("intField".attr, "stringField".attr)) + assertSuccess(EqualNullSafe("intField".attr, "stringField".attr)) + assertSuccess(LessThan("intField".attr, "stringField".attr)) + assertSuccess(LessThanOrEqual("intField".attr, "stringField".attr)) + assertSuccess(GreaterThan("intField".attr, "stringField".attr)) + assertSuccess(GreaterThanOrEqual("intField".attr, "stringField".attr)) // We will transform EqualTo with numeric and boolean types to CaseKeyWhen - assertSuccess(EqualTo(Symbol("intField"), Symbol("booleanField"))) - assertSuccess(EqualNullSafe(Symbol("intField"), Symbol("booleanField"))) + assertSuccess(EqualTo("intField".attr, "booleanField".attr)) + assertSuccess(EqualNullSafe("intField".attr, "booleanField".attr)) - assertErrorForDifferingTypes(EqualTo(Symbol("intField"), Symbol("mapField"))) - assertErrorForDifferingTypes(EqualNullSafe(Symbol("intField"), Symbol("mapField"))) - assertErrorForDifferingTypes(LessThan(Symbol("intField"), Symbol("booleanField"))) - assertErrorForDifferingTypes(LessThanOrEqual(Symbol("intField"), Symbol("booleanField"))) - assertErrorForDifferingTypes(GreaterThan(Symbol("intField"), Symbol("booleanField"))) - assertErrorForDifferingTypes(GreaterThanOrEqual(Symbol("intField"), Symbol("booleanField"))) + assertErrorForDifferingTypes(EqualTo("intField".attr, "mapField".attr)) + assertErrorForDifferingTypes(EqualNullSafe("intField".attr, "mapField".attr)) + assertErrorForDifferingTypes(LessThan("intField".attr, "booleanField".attr)) + assertErrorForDifferingTypes(LessThanOrEqual("intField".attr, "booleanField".attr)) + assertErrorForDifferingTypes(GreaterThan("intField".attr, "booleanField".attr)) + assertErrorForDifferingTypes(GreaterThanOrEqual("intField".attr, "booleanField".attr)) - assertError(EqualTo(Symbol("mapField"), Symbol("mapField")), + assertError(EqualTo("mapField".attr, "mapField".attr), "EqualTo does not support ordering on type map") - assertError(EqualNullSafe(Symbol("mapField"), Symbol("mapField")), + assertError(EqualNullSafe("mapField".attr, "mapField".attr), "EqualNullSafe does not support ordering on type map") - assertError(LessThan(Symbol("mapField"), Symbol("mapField")), + assertError(LessThan("mapField".attr, "mapField".attr), "LessThan does not support ordering on type map") - assertError(LessThanOrEqual(Symbol("mapField"), Symbol("mapField")), + assertError(LessThanOrEqual("mapField".attr, "mapField".attr), "LessThanOrEqual does not support ordering on type map") - assertError(GreaterThan(Symbol("mapField"), Symbol("mapField")), + assertError(GreaterThan("mapField".attr, "mapField".attr), "GreaterThan does not support ordering on type map") - assertError(GreaterThanOrEqual(Symbol("mapField"), Symbol("mapField")), + assertError(GreaterThanOrEqual("mapField".attr, "mapField".attr), "GreaterThanOrEqual does not support ordering on type map") - assertError(If(Symbol("intField"), Symbol("stringField"), Symbol("stringField")), + assertError(If("intField".attr, "stringField".attr, "stringField".attr), "type of predicate expression in If should be boolean") assertErrorForDifferingTypes( - If(Symbol("booleanField"), Symbol("intField"), Symbol("booleanField"))) + If("booleanField".attr, "intField".attr, "booleanField".attr)) assertError( - CaseWhen(Seq((Symbol("booleanField").attr, Symbol("intField").attr), - (Symbol("booleanField").attr, Symbol("mapField").attr))), + CaseWhen(Seq(("booleanField".attr, "intField".attr.attr), + ("booleanField".attr, "mapField".attr))), "THEN and ELSE expressions should all be same type or coercible to a common type") assertError( - CaseKeyWhen(Symbol("intField"), Seq(Symbol("intField"), Symbol("stringField"), - Symbol("intField"), Symbol("mapField"))), + CaseKeyWhen("intField".attr, Seq("intField".attr, "stringField".attr, + "intField".attr, "mapField".attr)), "THEN and ELSE expressions should all be same type or coercible to a common type") assertError( - CaseWhen(Seq((Symbol("booleanField").attr, Symbol("intField").attr), - (Symbol("intField").attr, Symbol("intField").attr))), + CaseWhen(Seq(("booleanField".attr, "intField".attr.attr), + ("intField".attr.attr, "intField".attr.attr))), "WHEN expressions in CaseWhen should all be boolean type") } @@ -150,29 +150,29 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { // instead of from AggregateExpression, which is the wrapper of an AggregateFunction. // We will cast String to Double for sum and average - assertSuccess(Sum(Symbol("stringField"))) - assertSuccess(Average(Symbol("stringField"))) - assertSuccess(Min(Symbol("arrayField"))) - assertSuccess(new BoolAnd(Symbol("booleanField"))) - assertSuccess(new BoolOr(Symbol("booleanField"))) - - assertError(Min(Symbol("mapField")), "min does not support ordering on type") - assertError(Max(Symbol("mapField")), "max does not support ordering on type") - assertError(Sum(Symbol("booleanField")), "function sum requires numeric type") - assertError(Average(Symbol("booleanField")), "function average requires numeric type") + assertSuccess(Sum("stringField".attr)) + assertSuccess(Average("stringField".attr)) + assertSuccess(Min("arrayField".attr)) + assertSuccess(new BoolAnd("booleanField".attr)) + assertSuccess(new BoolOr("booleanField".attr)) + + assertError(Min("mapField".attr), "min does not support ordering on type") + assertError(Max("mapField".attr), "max does not support ordering on type") + assertError(Sum("booleanField".attr), "function sum requires numeric type") + assertError(Average("booleanField".attr), "function average requires numeric type") } test("check types for others") { - assertError(CreateArray(Seq(Symbol("intField"), Symbol("booleanField"))), + assertError(CreateArray(Seq("intField".attr, "booleanField".attr)), "input to function array should all be the same type") - assertError(Coalesce(Seq(Symbol("intField"), Symbol("booleanField"))), + assertError(Coalesce(Seq("intField".attr, "booleanField".attr)), "input to function coalesce should all be the same type") assertError(Coalesce(Nil), "function coalesce requires at least one argument") assertError(new Murmur3Hash(Nil), "function hash requires at least one argument") assertError(new XxHash64(Nil), "function xxhash64 requires at least one argument") - assertError(Explode(Symbol("intField")), + assertError(Explode("intField".attr), "input to function explode should be array or map type") - assertError(PosExplode(Symbol("intField")), + assertError(PosExplode("intField".attr), "input to function explode should be array or map type") } @@ -183,7 +183,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { CreateNamedStruct(Seq(1, "a", "b", 2.0)), "Only foldable string expressions are allowed to appear at odd position") assertError( - CreateNamedStruct(Seq(Symbol("a").string.at(0), "a", "b", 2.0)), + CreateNamedStruct(Seq("a".attr.string.at(0), "a", "b", 2.0)), "Only foldable string expressions are allowed to appear at odd position") assertError( CreateNamedStruct(Seq(Literal.create(null, StringType), "a")), @@ -193,41 +193,41 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { test("check types for CreateMap") { assertError(CreateMap(Seq("a", "b", 2.0)), "even number of arguments") assertError( - CreateMap(Seq(Symbol("intField"), Symbol("stringField"), - Symbol("booleanField"), Symbol("stringField"))), + CreateMap(Seq("intField".attr, "stringField".attr, + "booleanField".attr, "stringField".attr)), "keys of function map should all be the same type") assertError( - CreateMap(Seq(Symbol("stringField"), Symbol("intField"), - Symbol("stringField"), Symbol("booleanField"))), + CreateMap(Seq("stringField".attr, "intField".attr, + "stringField".attr, "booleanField".attr)), "values of function map should all be the same type") } test("check types for ROUND/BROUND") { assertSuccess(Round(Literal(null), Literal(null))) - assertSuccess(Round(Symbol("intField"), Literal(1))) + assertSuccess(Round("intField".attr, Literal(1))) - assertError(Round(Symbol("intField"), Symbol("intField")), + assertError(Round("intField".attr, "intField".attr), "Only foldable Expression is allowed") - assertError(Round(Symbol("intField"), Symbol("booleanField")), "requires int type") - assertError(Round(Symbol("intField"), Symbol("mapField")), "requires int type") - assertError(Round(Symbol("booleanField"), Symbol("intField")), "requires numeric type") + assertError(Round("intField".attr, "booleanField".attr), "requires int type") + assertError(Round("intField".attr, "mapField".attr), "requires int type") + assertError(Round("booleanField".attr, "intField".attr), "requires numeric type") assertSuccess(BRound(Literal(null), Literal(null))) - assertSuccess(BRound(Symbol("intField"), Literal(1))) + assertSuccess(BRound("intField".attr, Literal(1))) - assertError(BRound(Symbol("intField"), Symbol("intField")), + assertError(BRound("intField".attr, "intField".attr), "Only foldable Expression is allowed") - assertError(BRound(Symbol("intField"), Symbol("booleanField")), "requires int type") - assertError(BRound(Symbol("intField"), Symbol("mapField")), "requires int type") - assertError(BRound(Symbol("booleanField"), Symbol("intField")), "requires numeric type") + assertError(BRound("intField".attr, "booleanField".attr), "requires int type") + assertError(BRound("intField".attr, "mapField".attr), "requires int type") + assertError(BRound("booleanField".attr, "intField".attr), "requires numeric type") } test("check types for Greatest/Least") { for (operator <- Seq[(Seq[Expression] => Expression)](Greatest, Least)) { - assertError(operator(Seq(Symbol("booleanField"))), "requires at least two arguments") - assertError(operator(Seq(Symbol("intField"), Symbol("stringField"))), + assertError(operator(Seq("booleanField".attr)), "requires at least two arguments") + assertError(operator(Seq("intField".attr, "stringField".attr)), "should all have the same type") - assertError(operator(Seq(Symbol("mapField"), Symbol("mapField"))), + assertError(operator(Seq("mapField".attr, "mapField".attr)), "does not support ordering") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/PullOutNondeterministicSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/PullOutNondeterministicSuite.scala index 72e10eadf79f..dd83c9cf40c2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/PullOutNondeterministicSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/PullOutNondeterministicSuite.scala @@ -27,10 +27,10 @@ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation */ class PullOutNondeterministicSuite extends AnalysisTest { - private lazy val a = 'a.int - private lazy val b = 'b.int + private lazy val a = "a".attr.int + private lazy val b = "b".attr.int private lazy val r = LocalRelation(a, b) - private lazy val rnd = Rand(10).as('_nondeterministic) + private lazy val rnd = Rand(10).as("_nondeterministic") private lazy val rndref = rnd.toAttribute test("no-op on filter") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupingAnalyticsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupingAnalyticsSuite.scala index cdfae1413829..e84fcfccaea5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupingAnalyticsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupingAnalyticsSuite.scala @@ -27,14 +27,14 @@ import org.apache.spark.sql.types._ class ResolveGroupingAnalyticsSuite extends AnalysisTest { - lazy val a = 'a.int - lazy val b = 'b.string - lazy val c = 'c.string + lazy val a = "a".attr.int + lazy val b = "b".attr.string + lazy val c = "c".attr.string lazy val unresolved_a = UnresolvedAttribute("a") lazy val unresolved_b = UnresolvedAttribute("b") lazy val unresolved_c = UnresolvedAttribute("c") - lazy val gid = 'spark_grouping_id.long.withNullability(false) - lazy val hive_gid = 'grouping__id.long.withNullability(false) + lazy val gid = "spark_grouping_id".attr.long.withNullability(false) + lazy val hive_gid = "grouping__id".attr.long.withNullability(false) lazy val grouping_a = Cast(ShiftRight(gid, 1) & 1L, ByteType, Option(TimeZone.getDefault().getID)) lazy val nulInt = Literal(null, IntegerType) lazy val nulStr = Literal(null, StringType) @@ -287,7 +287,7 @@ class ResolveGroupingAnalyticsSuite extends AnalysisTest { GroupingSets(Seq(Seq(), Seq(unresolved_a), Seq(unresolved_a, unresolved_b)), Seq(unresolved_a, unresolved_b), r1, Seq(unresolved_a, unresolved_b))) val expected = Project(Seq(a, b), Sort( - Seq(SortOrder('aggOrder.byte.withNullability(false), Ascending)), true, + Seq(SortOrder("aggOrder".attr.byte.withNullability(false), Ascending)), true, Aggregate(Seq(a, b, gid), Seq(a, b, grouping_a.as("aggOrder")), Expand( @@ -308,7 +308,7 @@ class ResolveGroupingAnalyticsSuite extends AnalysisTest { GroupingSets(Seq(Seq(), Seq(unresolved_a), Seq(unresolved_a, unresolved_b)), Seq(unresolved_a, unresolved_b), r1, Seq(unresolved_a, unresolved_b))) val expected3 = Project(Seq(a, b), Sort( - Seq(SortOrder('aggOrder.long.withNullability(false), Ascending)), true, + Seq(SortOrder("aggOrder".attr.long.withNullability(false), Ascending)), true, Aggregate(Seq(a, b, gid), Seq(a, b, gid.as("aggOrder")), Expand( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala index 4b9a2ca94ea5..3652f2effb24 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala @@ -72,8 +72,8 @@ class ResolveHintsSuite extends AnalysisTest { test("do not traverse past existing broadcast hints") { checkAnalysisWithoutViewWrapper( UnresolvedHint("MAPJOIN", Seq("table"), - ResolvedHint(table("table").where('a > 1), HintInfo(strategy = Some(BROADCAST)))), - ResolvedHint(testRelation.where('a > 1), HintInfo(strategy = Some(BROADCAST))).analyze, + ResolvedHint(table("table").where("a".attr > 1), HintInfo(strategy = Some(BROADCAST)))), + ResolvedHint(testRelation.where("a".attr > 1), HintInfo(strategy = Some(BROADCAST))).analyze, caseSensitive = false) } @@ -84,7 +84,7 @@ class ResolveHintsSuite extends AnalysisTest { caseSensitive = false) checkAnalysisWithoutViewWrapper( - UnresolvedHint("MAPJOIN", Seq("tableAlias"), table("table").subquery('tableAlias)), + UnresolvedHint("MAPJOIN", Seq("tableAlias"), table("table").subquery("tableAlias")), ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) @@ -97,8 +97,9 @@ class ResolveHintsSuite extends AnalysisTest { test("do not traverse past subquery alias") { checkAnalysisWithoutViewWrapper( - UnresolvedHint("MAPJOIN", Seq("table"), table("table").where('a > 1).subquery('tableAlias)), - testRelation.where('a > 1).analyze, + UnresolvedHint("MAPJOIN", Seq("table"), + table("table").where("a".attr > 1).subquery("tableAlias")), + testRelation.where("a".attr > 1).analyze, caseSensitive = false) } @@ -110,8 +111,9 @@ class ResolveHintsSuite extends AnalysisTest { |SELECT /*+ BROADCAST(ctetable) */ * FROM ctetable """.stripMargin ), - ResolvedHint(testRelation.where('a > 1).select('a), HintInfo(strategy = Some(BROADCAST))) - .select('a).analyze, + ResolvedHint(testRelation.where("a".attr > 1).select("a".attr), + HintInfo(strategy = Some(BROADCAST))) + .select("a".attr).analyze, caseSensitive = false) } @@ -123,7 +125,7 @@ class ResolveHintsSuite extends AnalysisTest { |SELECT /*+ BROADCAST(table) */ * FROM ctetable """.stripMargin ), - testRelation.where('a > 1).select('a).select('a).analyze, + testRelation.where("a".attr > 1).select("a".attr).select("a".attr).analyze, caseSensitive = false) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala index b9233a27f3d7..9b5cf9e7d654 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala @@ -35,9 +35,9 @@ class ResolveLambdaVariablesSuite extends PlanTest { val batches = Batch("Resolution", FixedPoint(4), ResolveLambdaVariables) :: Nil } - private val key = 'key.int - private val values1 = 'values1.array(IntegerType) - private val values2 = 'values2.array(ArrayType(ArrayType(IntegerType))) + private val key = "key".attr.int + private val values1 = "values1".attr.array(IntegerType) + private val values2 = "values2".attr.array(ArrayType(ArrayType(IntegerType))) private val data = LocalRelation(Seq(key, values1, values2)) private val lvInt = NamedLambdaVariable("x", IntegerType, nullable = true) private val lvHiddenInt = NamedLambdaVariable("col0", IntegerType, nullable = true) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveNaturalJoinSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveNaturalJoinSuite.scala index ea2284e5420b..dacc150e401a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveNaturalJoinSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveNaturalJoinSuite.scala @@ -24,10 +24,10 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation class ResolveNaturalJoinSuite extends AnalysisTest { - lazy val a = 'a.string - lazy val b = 'b.string - lazy val c = 'c.string - lazy val d = 'd.struct('f1.int, 'f2.long) + lazy val a = "a".attr.string + lazy val b = "b".attr.string + lazy val c = "c".attr.string + lazy val d = "d".attr.struct("f1".attr.int, "f2".attr.long) lazy val aNotNull = a.notNull lazy val bNotNull = b.notNull lazy val cNotNull = c.notNull diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala index 5aa80e1a9bd7..095f5bc40976 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala @@ -28,9 +28,9 @@ import org.apache.spark.sql.catalyst.plans.logical._ */ class ResolveSubquerySuite extends AnalysisTest { - val a = 'a.int - val b = 'b.int - val c = 'c.int + val a = "a".attr.int + val b = "b".attr.int + val c = "c".attr.int val t1 = LocalRelation(a) val t2 = LocalRelation(b) val t3 = LocalRelation(c) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolvedUuidExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolvedUuidExpressionsSuite.scala index 5ddfa9f2191e..2528f2174e1b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolvedUuidExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolvedUuidExpressionsSuite.scala @@ -28,11 +28,11 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} */ class ResolvedUuidExpressionsSuite extends AnalysisTest { - private lazy val a = 'a.int + private lazy val a = "a".attr.int private lazy val r = LocalRelation(a) - private lazy val uuid1 = Uuid().as('_uuid1) - private lazy val uuid2 = Uuid().as('_uuid2) - private lazy val uuid3 = Uuid().as('_uuid3) + private lazy val uuid1 = Uuid().as("uuid1") + private lazy val uuid2 = Uuid().as("uuid2") + private lazy val uuid3 = Uuid().as("uuid3") private lazy val uuid1Ref = uuid1.toAttribute private val tracker = new QueryPlanningTracker diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinalsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinalsSuite.scala index c0312282c76c..534b76ab9a72 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinalsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinalsSuite.scala @@ -52,10 +52,10 @@ class SubstituteUnresolvedOrdinalsSuite extends AnalysisTest { test("group by ordinal") { // Tests group by ordinal, apply single rule. - val plan2 = testRelation2.groupBy(Literal(1), Literal(2))('a, 'b) + val plan2 = testRelation2.groupBy(Literal(1), Literal(2))("a".attr, "b".attr) comparePlans( SubstituteUnresolvedOrdinals.apply(plan2), - testRelation2.groupBy(UnresolvedOrdinal(1), UnresolvedOrdinal(2))('a, 'b)) + testRelation2.groupBy(UnresolvedOrdinal(1), UnresolvedOrdinal(2))("a".attr, "b".attr)) // Tests group by ordinal, do full analysis checkAnalysis(plan2, testRelation2.groupBy(a, b)(a, b)) @@ -64,7 +64,7 @@ class SubstituteUnresolvedOrdinalsSuite extends AnalysisTest { withSQLConf(SQLConf.GROUP_BY_ORDINAL.key -> "false") { comparePlans( SubstituteUnresolvedOrdinals.apply(plan2), - testRelation2.groupBy(Literal(1), Literal(2))('a, 'b)) + testRelation2.groupBy(Literal(1), Literal(2))("a".attr, "b".attr)) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index d310538e302d..863478460763 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -498,18 +498,18 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac val tbl2 = catalog.getTable("db2", "tbl2") checkAnswer(tbl2, Seq.empty, Set(part1, part2)) - checkAnswer(tbl2, Seq('a.int <= 1), Set(part1)) - checkAnswer(tbl2, Seq('a.int === 2), Set.empty) - checkAnswer(tbl2, Seq(In('a.int * 10, Seq(30))), Set(part2)) - checkAnswer(tbl2, Seq(Not(In('a.int, Seq(4)))), Set(part1, part2)) - checkAnswer(tbl2, Seq('a.int === 1, 'b.string === "2"), Set(part1)) - checkAnswer(tbl2, Seq('a.int === 1 && 'b.string === "2"), Set(part1)) - checkAnswer(tbl2, Seq('a.int === 1, 'b.string === "x"), Set.empty) - checkAnswer(tbl2, Seq('a.int === 1 || 'b.string === "x"), Set(part1)) + checkAnswer(tbl2, Seq("a".attr.int <= 1), Set(part1)) + checkAnswer(tbl2, Seq("a".attr.int === 2), Set.empty) + checkAnswer(tbl2, Seq(In("a".attr.int * 10, Seq(30))), Set(part2)) + checkAnswer(tbl2, Seq(Not(In("a".attr.int, Seq(4)))), Set(part1, part2)) + checkAnswer(tbl2, Seq("a".attr.int === 1, "b".attr.string === "2"), Set(part1)) + checkAnswer(tbl2, Seq("a".attr.int === 1 && "b".attr.string === "2"), Set(part1)) + checkAnswer(tbl2, Seq("a".attr.int === 1, "b".attr.string === "x"), Set.empty) + checkAnswer(tbl2, Seq("a".attr.int === 1 || "b".attr.string === "x"), Set(part1)) intercept[AnalysisException] { try { - checkAnswer(tbl2, Seq('a.int > 0 && 'col1.int > 0), Set.empty) + checkAnswer(tbl2, Seq("a".attr.int > 0 && "col1".attr.int > 0), Set.empty) } catch { // HiveExternalCatalog may be the first one to notice and throw an exception, which will // then be caught and converted to a RuntimeException with a descriptive message. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala index 577814b9c669..e9da2e244030 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala @@ -54,17 +54,17 @@ class EncoderResolutionSuite extends PlanTest { val encoder = ExpressionEncoder[StringLongClass] // int type can be up cast to long type - val attrs1 = Seq('a.string, 'b.int) + val attrs1 = Seq("a".attr.string, "b".attr.int) testFromRow(encoder, attrs1, InternalRow(str, 1)) // int type can be up cast to string type - val attrs2 = Seq('a.int, 'b.long) + val attrs2 = Seq("a".attr.int, "b".attr.long) testFromRow(encoder, attrs2, InternalRow(1, 2L)) } test("real type doesn't match encoder schema but they are compatible: nested product") { val encoder = ExpressionEncoder[ComplexClass] - val attrs = Seq('a.int, 'b.struct('a.int, 'b.long)) + val attrs = Seq("a".attr.int, "b".attr.struct("a".attr.int, "b".attr.long)) testFromRow(encoder, attrs, InternalRow(1, InternalRow(2, 3L))) } @@ -72,20 +72,20 @@ class EncoderResolutionSuite extends PlanTest { val encoder = ExpressionEncoder.tuple( ExpressionEncoder[StringLongClass], ExpressionEncoder[Long]) - val attrs = Seq('a.struct('a.string, 'b.byte), 'b.int) + val attrs = Seq("a".attr.struct("a".attr.string, "b".attr.byte), "b".attr.int) testFromRow(encoder, attrs, InternalRow(InternalRow(str, 1.toByte), 2)) } test("real type doesn't match encoder schema but they are compatible: primitive array") { val encoder = ExpressionEncoder[PrimitiveArrayClass] - val attrs = Seq('arr.array(IntegerType)) + val attrs = Seq("arr".attr.array(IntegerType)) val array = new GenericArrayData(Array(1, 2, 3)) testFromRow(encoder, attrs, InternalRow(array)) } test("the real type is not compatible with encoder schema: primitive array") { val encoder = ExpressionEncoder[PrimitiveArrayClass] - val attrs = Seq('arr.array(StringType)) + val attrs = Seq("arr".attr.array(StringType)) assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == s""" |Cannot up cast array element from string to bigint. @@ -99,7 +99,8 @@ class EncoderResolutionSuite extends PlanTest { test("real type doesn't match encoder schema but they are compatible: array") { val encoder = ExpressionEncoder[ArrayClass] - val attrs = Seq('arr.array(new StructType().add("a", "int").add("b", "int").add("c", "int"))) + val attrs = + Seq("arr".attr.array(new StructType().add("a", "int").add("b", "int").add("c", "int"))) val array = new GenericArrayData(Array(InternalRow(1, 2, 3))) testFromRow(encoder, attrs, InternalRow(array)) } @@ -108,7 +109,7 @@ class EncoderResolutionSuite extends PlanTest { val encoder = ExpressionEncoder[NestedArrayClass] val et = new StructType().add("arr", ArrayType( new StructType().add("a", "int").add("b", "int").add("c", "int"))) - val attrs = Seq('nestedArr.array(et)) + val attrs = Seq("nestedArr".attr.array(et)) val innerArr = new GenericArrayData(Array(InternalRow(1, 2, 3))) val outerArr = new GenericArrayData(Array(InternalRow(innerArr))) testFromRow(encoder, attrs, InternalRow(outerArr)) @@ -116,14 +117,14 @@ class EncoderResolutionSuite extends PlanTest { test("the real type is not compatible with encoder schema: non-array field") { val encoder = ExpressionEncoder[ArrayClass] - val attrs = Seq('arr.int) + val attrs = Seq("arr".attr.int) assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == "need an array field but got int") } test("the real type is not compatible with encoder schema: array element type") { val encoder = ExpressionEncoder[ArrayClass] - val attrs = Seq('arr.array(new StructType().add("c", "int"))) + val attrs = Seq("arr".attr.array(new StructType().add("c", "int"))) assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == "No such struct field a in c") } @@ -132,13 +133,13 @@ class EncoderResolutionSuite extends PlanTest { val encoder = ExpressionEncoder[NestedArrayClass] withClue("inner element is not array") { - val attrs = Seq('nestedArr.array(new StructType().add("arr", "int"))) + val attrs = Seq("nestedArr".attr.array(new StructType().add("arr", "int"))) assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == "need an array field but got int") } withClue("nested array element type is not compatible") { - val attrs = Seq('nestedArr.array(new StructType() + val attrs = Seq("nestedArr".attr.array(new StructType() .add("arr", ArrayType(new StructType().add("c", "int"))))) assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == "No such struct field a in c") @@ -147,7 +148,7 @@ class EncoderResolutionSuite extends PlanTest { test("nullability of array type element should not fail analysis") { val encoder = ExpressionEncoder[Seq[Int]] - val attrs = 'a.array(IntegerType) :: Nil + val attrs = "a".attr.array(IntegerType) :: Nil // It should pass analysis val fromRow = encoder.resolveAndBind(attrs).createDeserializer() @@ -166,14 +167,14 @@ class EncoderResolutionSuite extends PlanTest { val encoder = ExpressionEncoder[(String, Long)] { - val attrs = Seq('a.string, 'b.long, 'c.int) + val attrs = Seq("a".attr.string, "b".attr.long, "c".attr.int) assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == "Try to map struct to Tuple2, " + "but failed as the number of fields does not line up.") } { - val attrs = Seq('a.string) + val attrs = Seq("a".attr.string) assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == "Try to map struct to Tuple2, " + "but failed as the number of fields does not line up.") @@ -184,14 +185,15 @@ class EncoderResolutionSuite extends PlanTest { val encoder = ExpressionEncoder[(String, (Long, String))] { - val attrs = Seq('a.string, 'b.struct('x.long, 'y.string, 'z.int)) + val attrs = + Seq("a".attr.string, "b".attr.struct("x".attr.long, "y".attr.string, "z".attr.int)) assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == "Try to map struct to Tuple2, " + "but failed as the number of fields does not line up.") } { - val attrs = Seq('a.string, 'b.struct('x.long)) + val attrs = Seq("a".attr.string, "b".attr.struct("x".attr.long)) assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == "Try to map struct to Tuple2, " + "but failed as the number of fields does not line up.") @@ -200,14 +202,16 @@ class EncoderResolutionSuite extends PlanTest { test("nested case class can have different number of fields from the real schema") { val encoder = ExpressionEncoder[(String, StringIntClass)] - val attrs = Seq('a.string, 'b.struct('a.string, 'b.int, 'c.int)) + val attrs = + Seq("a".attr.string, "b".attr.struct("a".attr.string, "b".attr.int, "c".attr.int)) encoder.resolveAndBind(attrs) } test("SPARK-28497: complex type is not compatible with string encoder schema") { val encoder = ExpressionEncoder[String] - Seq('a.struct('x.long), 'a.array(StringType), 'a.map(StringType, StringType)).foreach { attr => + Seq("a".attr.struct("x".attr.long), "a".attr.array(StringType), + "a".attr.mapAttr(StringType, StringType)).foreach { attr => val attrs = Seq(attr) assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == s""" @@ -221,7 +225,7 @@ class EncoderResolutionSuite extends PlanTest { test("throw exception if real type is not compatible with encoder schema") { val msg1 = intercept[AnalysisException] { - ExpressionEncoder[StringIntClass].resolveAndBind(Seq('a.string, 'b.long)) + ExpressionEncoder[StringIntClass].resolveAndBind(Seq("a".attr.string, "b".attr.long)) }.message assert(msg1 == s""" @@ -234,7 +238,8 @@ class EncoderResolutionSuite extends PlanTest { val msg2 = intercept[AnalysisException] { val structType = new StructType().add("a", StringType).add("b", DecimalType.SYSTEM_DEFAULT) - ExpressionEncoder[ComplexClass].resolveAndBind(Seq('a.long, 'b.struct(structType))) + ExpressionEncoder[ComplexClass].resolveAndBind( + Seq("a".attr.long, "b".attr.struct(structType))) }.message assert(msg2 == s""" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala index 14dd04afebe2..fb884777a73d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala @@ -291,11 +291,11 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper test("function least") { val row = create_row(1, 2, "a", "b", "c") - val c1 = 'a.int.at(0) - val c2 = 'a.int.at(1) - val c3 = 'a.string.at(2) - val c4 = 'a.string.at(3) - val c5 = 'a.string.at(4) + val c1 = "a".attr.int.at(0) + val c2 = "a".attr.int.at(1) + val c3 = "a".attr.string.at(2) + val c4 = "a".attr.string.at(3) + val c5 = "a".attr.string.at(4) checkEvaluation(Least(Seq(c4, c3, c5)), "a", row) checkEvaluation(Least(Seq(c1, c2)), 1, row) checkEvaluation(Least(Seq(c1, c2, Literal(-1))), -1, row) @@ -348,11 +348,11 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper test("function greatest") { val row = create_row(1, 2, "a", "b", "c") - val c1 = 'a.int.at(0) - val c2 = 'a.int.at(1) - val c3 = 'a.string.at(2) - val c4 = 'a.string.at(3) - val c5 = 'a.string.at(4) + val c1 = "a".attr.int.at(0) + val c2 = "a".attr.int.at(1) + val c3 = "a".attr.string.at(2) + val c4 = "a".attr.string.at(3) + val c5 = "a".attr.string.at(4) checkEvaluation(Greatest(Seq(c4, c5, c3)), "c", row) checkEvaluation(Greatest(Seq(c2, c1)), 2, row) checkEvaluation(Greatest(Seq(c1, c2, Literal(2))), 2, row) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/BitwiseExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/BitwiseExpressionsSuite.scala index 718d8dd44321..bf635bde4149 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/BitwiseExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/BitwiseExpressionsSuite.scala @@ -151,11 +151,11 @@ class BitwiseExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val row4 = create_row(11.toShort, 16) val row5 = create_row(11.toByte, 16) - val tl = 't.long.at(0) - val ti = 't.int.at(0) - val ts = 't.short.at(0) - val tb = 't.byte.at(0) - val p = 'p.int.at(1) + val tl = "t".attr.long.at(0) + val ti = "t".attr.int.at(0) + val ts = "t".attr.short.at(0) + val tb = "t".attr.byte.at(0) + val p = "p".attr.int.at(1) val expr = BitwiseGet(tl, p) checkExceptionInExpression[IllegalArgumentException]( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala index ac31a68b2b61..64e6f678cb2c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala @@ -99,9 +99,9 @@ class CanonicalizeSuite extends SparkFunSuite { test("SPARK-32927: Bitwise operations are commutative") { Seq(BitwiseOr(_, _), BitwiseAnd(_, _), BitwiseXor(_, _)).foreach { f => - val e1 = f('a, f('b, 'c)) - val e2 = f(f('a, 'b), 'c) - val e3 = f('a, f('b, 'a)) + val e1 = f("a".attr, f("b".attr, "c".attr)) + val e2 = f(f("a".attr, "b".attr), "c".attr) + val e3 = f("a".attr, f("b".attr, "a".attr)) assert(e1.canonicalized == e2.canonicalized) assert(e1.canonicalized != e3.canonicalized) @@ -110,9 +110,9 @@ class CanonicalizeSuite extends SparkFunSuite { test("SPARK-32927: Bitwise operations are commutative for non-deterministic expressions") { Seq(BitwiseOr(_, _), BitwiseAnd(_, _), BitwiseXor(_, _)).foreach { f => - val e1 = f('a, f(rand(42), 'c)) - val e2 = f(f('a, rand(42)), 'c) - val e3 = f('a, f(rand(42), 'a)) + val e1 = f("a".attr, f(rand(42), "c".attr)) + val e2 = f(f("a".attr, rand(42)), "c".attr) + val e3 = f("a".attr, f(rand(42), "a".attr)) assert(e1.canonicalized == e2.canonicalized) assert(e1.canonicalized != e3.canonicalized) @@ -121,9 +121,9 @@ class CanonicalizeSuite extends SparkFunSuite { test("SPARK-32927: Bitwise operations are commutative for literal expressions") { Seq(BitwiseOr(_, _), BitwiseAnd(_, _), BitwiseXor(_, _)).foreach { f => - val e1 = f('a, f(42, 'c)) - val e2 = f(f('a, 42), 'c) - val e3 = f('a, f(42, 'a)) + val e1 = f("a".attr, f(42, "c".attr)) + val e2 = f(f("a".attr, 42), "c".attr) + val e3 = f("a".attr, f(42, "a".attr)) assert(e1.canonicalized == e2.canonicalized) assert(e1.canonicalized != e3.canonicalized) @@ -133,9 +133,9 @@ class CanonicalizeSuite extends SparkFunSuite { test("SPARK-32927: Bitwise operations are commutative in a complex case") { Seq(BitwiseOr(_, _), BitwiseAnd(_, _), BitwiseXor(_, _)).foreach { f1 => Seq(BitwiseOr(_, _), BitwiseAnd(_, _), BitwiseXor(_, _)).foreach { f2 => - val e1 = f2(f1('a, f1('b, 'c)), 'a) - val e2 = f2(f1(f1('a, 'b), 'c), 'a) - val e3 = f2(f1('a, f1('b, 'a)), 'a) + val e1 = f2(f1("a".attr, f1("b".attr, "c".attr)), "a".attr) + val e2 = f2(f1(f1("a".attr, "b".attr), "c".attr), "a".attr) + val e3 = f2(f1("a".attr, f1("b".attr, "a".attr)), "a".attr) assert(e1.canonicalized == e2.canonicalized) assert(e1.canonicalized != e3.canonicalized) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala index 57abdb4de229..eb76abe98ddc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala @@ -386,16 +386,16 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { test("CreateStruct") { val row = create_row(1, 2, 3) - val c1 = 'a.int.at(0) - val c3 = 'c.int.at(2) + val c1 = "a".attr.int.at(0) + val c3 = "c".attr.int.at(2) checkEvaluation(CreateStruct(Seq(c1, c3)), create_row(1, 3), row) checkEvaluation(CreateStruct(Literal.create(null, LongType) :: Nil), create_row(null)) } test("CreateNamedStruct") { val row = create_row(1, 2, 3) - val c1 = 'a.int.at(0) - val c3 = 'c.int.at(2) + val c1 = "a".attr.int.at(0) + val c3 = "c".attr.int.at(2) checkEvaluation(CreateNamedStruct(Seq("a", c1, "b", c3)), create_row(1, 3), row) checkEvaluation(CreateNamedStruct(Seq("a", c1, "b", "y")), create_row(1, UTF8String.fromString("y")), row) @@ -410,11 +410,12 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { ExtractValue(u.child, u.extraction, _ == _) } - checkEvaluation(quickResolve('c.map(MapType(StringType, StringType)).at(0).getItem("a")), + checkEvaluation( + quickResolve("c".attr.mapAttr(MapType(StringType, StringType)).at(0).getItem("a")), "b", create_row(Map("a" -> "b"))) - checkEvaluation(quickResolve('c.array(StringType).at(0).getItem(1)), + checkEvaluation(quickResolve("c".attr.array(StringType).at(0).getItem(1)), "b", create_row(Seq("a", "b"))) - checkEvaluation(quickResolve('c.struct('a.int).at(0).getField("a")), + checkEvaluation(quickResolve("c".attr.struct("a".attr.int).at(0).getField("a")), 1, create_row(create_row(1))) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala index ee6f89a155ae..ad913f604218 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala @@ -72,12 +72,12 @@ class ConditionalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper test("case when") { val row = create_row(null, false, true, "a", "b", "c") - val c1 = 'a.boolean.at(0) - val c2 = 'a.boolean.at(1) - val c3 = 'a.boolean.at(2) - val c4 = 'a.string.at(3) - val c5 = 'a.string.at(4) - val c6 = 'a.string.at(5) + val c1 = "a".attr.boolean.at(0) + val c2 = "a".attr.boolean.at(1) + val c3 = "a".attr.boolean.at(2) + val c4 = "a".attr.string.at(3) + val c5 = "a".attr.string.at(4) + val c6 = "a".attr.string.at(5) checkEvaluation(CaseWhen(Seq((c1, c4)), c6), "c", row) checkEvaluation(CaseWhen(Seq((c2, c4)), c6), "c", row) @@ -95,9 +95,9 @@ class ConditionalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper assert(CaseWhen(Seq((c2, c4), (c3, c5)), c6).nullable) assert(CaseWhen(Seq((c2, c4), (c3, c5))).nullable) - val c4_notNull = 'a.boolean.notNull.at(3) - val c5_notNull = 'a.boolean.notNull.at(4) - val c6_notNull = 'a.boolean.notNull.at(5) + val c4_notNull = "a".attr.boolean.notNull.at(3) + val c5_notNull = "a".attr.boolean.notNull.at(4) + val c6_notNull = "a".attr.boolean.notNull.at(5) assert(CaseWhen(Seq((c2, c4_notNull)), c6_notNull).nullable === false) assert(CaseWhen(Seq((c2, c4)), c6_notNull).nullable) @@ -143,8 +143,10 @@ class ConditionalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val arrayCaseWhen2 = CaseWhen(Seq((Literal.FalseLiteral, arrayWithoutNulls)), arrayWithNulls) val arrayCaseWhen3 = CaseWhen(Seq((Literal.TrueLiteral, arrayWithNulls)), arrayWithoutNulls) val arrayCaseWhen4 = CaseWhen(Seq((Literal.TrueLiteral, arrayWithoutNulls)), arrayWithNulls) - val structCaseWhen1 = CaseWhen(Seq((Literal.FalseLiteral, structWithNulls)), structWithoutNulls) - val structCaseWhen2 = CaseWhen(Seq((Literal.FalseLiteral, structWithoutNulls)), structWithNulls) + val structCaseWhen1 = + CaseWhen(Seq((Literal.FalseLiteral, structWithNulls)), structWithoutNulls) + val structCaseWhen2 = + CaseWhen(Seq((Literal.FalseLiteral, structWithoutNulls)), structWithNulls) val structCaseWhen3 = CaseWhen(Seq((Literal.TrueLiteral, structWithNulls)), structWithoutNulls) val structCaseWhen4 = CaseWhen(Seq((Literal.TrueLiteral, structWithoutNulls)), structWithNulls) val mapCaseWhen1 = CaseWhen(Seq((Literal.FalseLiteral, mapWithNulls)), mapWithoutNulls) @@ -186,12 +188,12 @@ class ConditionalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper test("case key when") { val row = create_row(null, 1, 2, "a", "b", "c") - val c1 = 'a.int.at(0) - val c2 = 'a.int.at(1) - val c3 = 'a.int.at(2) - val c4 = 'a.string.at(3) - val c5 = 'a.string.at(4) - val c6 = 'a.string.at(5) + val c1 = "a".attr.int.at(0) + val c2 = "a".attr.int.at(1) + val c3 = "a".attr.int.at(2) + val c4 = "a".attr.string.at(3) + val c5 = "a".attr.string.at(4) + val c6 = "a".attr.string.at(5) val literalNull = Literal.create(null, IntegerType) val literalInt = Literal(1) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala index c12dd3051d27..c1b37a48a753 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala @@ -94,72 +94,73 @@ class ExpressionSQLBuilderSuite extends SparkFunSuite { } test("attributes") { - checkSQL('a.int, "`a`") - checkSQL(Symbol("foo bar").int, "`foo bar`") + checkSQL("a".attr.int, "`a`") + checkSQL("foo bar".attr.int, "`foo bar`") // Keyword - checkSQL('int.int, "`int`") + checkSQL("int".attr.int, "`int`") } test("binary comparisons") { - checkSQL('a.int === 'b.int, "(`a` = `b`)") - checkSQL('a.int <=> 'b.int, "(`a` <=> `b`)") - checkSQL('a.int =!= 'b.int, "(NOT (`a` = `b`))") + checkSQL("a".attr.int === "b".attr.int, "(`a` = `b`)") + checkSQL("a".attr.int <=> "b".attr.int, "(`a` <=> `b`)") + checkSQL("a".attr.int =!= "b".attr.int, "(NOT (`a` = `b`))") - checkSQL('a.int < 'b.int, "(`a` < `b`)") - checkSQL('a.int <= 'b.int, "(`a` <= `b`)") - checkSQL('a.int > 'b.int, "(`a` > `b`)") - checkSQL('a.int >= 'b.int, "(`a` >= `b`)") + checkSQL("a".attr.int < "b".attr.int, "(`a` < `b`)") + checkSQL("a".attr.int <= "b".attr.int, "(`a` <= `b`)") + checkSQL("a".attr.int > "b".attr.int, "(`a` > `b`)") + checkSQL("a".attr.int >= "b".attr.int, "(`a` >= `b`)") - checkSQL('a.int in ('b.int, 'c.int), "(`a` IN (`b`, `c`))") - checkSQL('a.int in (1, 2), "(`a` IN (1, 2))") + checkSQL("a".attr.int in ("b".attr.int, "c".attr.int), "(`a` IN (`b`, `c`))") + checkSQL("a".attr.int in (1, 2), "(`a` IN (1, 2))") - checkSQL('a.int.isNull, "(`a` IS NULL)") - checkSQL('a.int.isNotNull, "(`a` IS NOT NULL)") + checkSQL("a".attr.int.isNull, "(`a` IS NULL)") + checkSQL("a".attr.int.isNotNull, "(`a` IS NOT NULL)") } test("logical operators") { - checkSQL('a.boolean && 'b.boolean, "(`a` AND `b`)") - checkSQL('a.boolean || 'b.boolean, "(`a` OR `b`)") - checkSQL(!'a.boolean, "(NOT `a`)") - checkSQL(If('a.boolean, 'b.int, 'c.int), "(IF(`a`, `b`, `c`))") + checkSQL("a".attr.boolean && "b".attr.boolean, "(`a` AND `b`)") + checkSQL("a".attr.boolean || "b".attr.boolean, "(`a` OR `b`)") + checkSQL(!"a".attr.boolean, "(NOT `a`)") + checkSQL(If("a".attr.boolean, "b".attr.int, "c".attr.int), "(IF(`a`, `b`, `c`))") } test("arithmetic expressions") { - checkSQL('a.int + 'b.int, "(`a` + `b`)") - checkSQL('a.int - 'b.int, "(`a` - `b`)") - checkSQL('a.int * 'b.int, "(`a` * `b`)") - checkSQL('a.int / 'b.int, "(`a` / `b`)") - checkSQL('a.int % 'b.int, "(`a` % `b`)") - - checkSQL(-'a.int, "(- `a`)") - checkSQL(-('a.int + 'b.int), "(- (`a` + `b`))") + checkSQL("a".attr.int + "b".attr.int, "(`a` + `b`)") + checkSQL("a".attr.int - "b".attr.int, "(`a` - `b`)") + checkSQL("a".attr.int * "b".attr.int, "(`a` * `b`)") + checkSQL("a".attr.int / "b".attr.int, "(`a` / `b`)") + checkSQL("a".attr.int % "b".attr.int, "(`a` % `b`)") + + checkSQL(-"a".attr.int, "(- `a`)") + checkSQL(-("a".attr.int + "b".attr.int), "(- (`a` + `b`))") } test("window specification") { val frame = SpecifiedWindowFrame(RangeFrame, UnboundedPreceding, CurrentRow) checkSQL( - WindowSpecDefinition('a.int :: Nil, Nil, frame), + WindowSpecDefinition("a".attr.int :: Nil, Nil, frame), s"(PARTITION BY `a` ${frame.sql})" ) checkSQL( - WindowSpecDefinition('a.int :: 'b.string :: Nil, Nil, frame), + WindowSpecDefinition("a".attr.int :: "b".attr.string :: Nil, Nil, frame), s"(PARTITION BY `a`, `b` ${frame.sql})" ) checkSQL( - WindowSpecDefinition(Nil, 'a.int.asc :: Nil, frame), + WindowSpecDefinition(Nil, "a".attr.int.asc :: Nil, frame), s"(ORDER BY `a` ASC NULLS FIRST ${frame.sql})" ) checkSQL( - WindowSpecDefinition(Nil, 'a.int.asc :: 'b.string.desc :: Nil, frame), + WindowSpecDefinition(Nil, "a".attr.int.asc :: "b".attr.string.desc :: Nil, frame), s"(ORDER BY `a` ASC NULLS FIRST, `b` DESC NULLS LAST ${frame.sql})" ) checkSQL( - WindowSpecDefinition('a.int :: 'b.string :: Nil, 'c.int.asc :: 'd.string.desc :: Nil, frame), + WindowSpecDefinition("a".attr.int :: "b".attr.string :: Nil, + "c".attr.int.asc :: "d".attr.string.desc :: Nil, frame), s"(PARTITION BY `a`, `b` ORDER BY `c` ASC NULLS FIRST, `d` DESC NULLS LAST ${frame.sql})" ) } @@ -168,17 +169,17 @@ class ExpressionSQLBuilderSuite extends SparkFunSuite { val interval = Literal(new CalendarInterval(0, 0, MICROS_PER_HOUR)) checkSQL( - TimeAdd('a, interval), + TimeAdd("a".attr, interval), "`a` + INTERVAL '1 hours'" ) checkSQL( - DatetimeSub('a, interval, Literal.default(TimestampType)), + DatetimeSub("a".attr, interval, Literal.default(TimestampType)), "`a` - INTERVAL '1 hours'" ) checkSQL( - DateAddInterval('a, interval), + DateAddInterval("a".attr, interval), "`a` + INTERVAL '1 hours'" ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala index 72d15e8abef6..bcc053050ac0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala @@ -410,11 +410,11 @@ class MathExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { testUnary(Bin, java.lang.Long.toBinaryString, (-20 to 20).map(_.toLong), evalType = LongType) val row = create_row(null, 12L, 123L, 1234L, -123L) - val l1 = 'a.long.at(0) - val l2 = 'a.long.at(1) - val l3 = 'a.long.at(2) - val l4 = 'a.long.at(3) - val l5 = 'a.long.at(4) + val l1 = "a".attr.long.at(0) + val l2 = "a".attr.long.at(1) + val l3 = "a".attr.long.at(2) + val l4 = "a".attr.long.at(3) + val l5 = "a".attr.long.at(4) checkEvaluation(Bin(l1), null, row) checkEvaluation(Bin(l2), java.lang.Long.toBinaryString(12), row) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala index d425d0ba4218..00c6378cf9ce 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala @@ -464,7 +464,7 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // with dummy input, resolve the plan by the analyzer, and replace the dummy input // with a literal for tests. val unresolvedDeser = UnresolvedDeserializer(encoderFor[Map[Int, String]].deserializer) - val dummyInputPlan = LocalRelation('value.map(MapType(IntegerType, StringType))) + val dummyInputPlan = LocalRelation("value".attr.mapAttr(MapType(IntegerType, StringType))) val plan = Project(Alias(unresolvedDeser, "none")() :: Nil, dummyInputPlan) val analyzedPlan = SimpleAnalyzer.execute(plan) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala index 8d7501d952ec..f88fa12cfd09 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala @@ -44,7 +44,7 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { (implicit inputToExpression: A => Expression): Unit = { checkEvaluation(mkExpr(input), expected) // check literal input - val regex = 'a.string.at(0) + val regex = "a".attr.string.at(0) checkEvaluation(mkExpr(regex), expected, create_row(input)) // check row input } @@ -279,7 +279,7 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { evaluateWithoutCodegen("abbbbc" rlike "**") } intercept[java.util.regex.PatternSyntaxException] { - val regex = 'a.string.at(0) + val regex = "a".attr.string.at(0) evaluateWithoutCodegen("abbbbc" rlike regex, create_row("**")) } } @@ -292,9 +292,9 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val row5 = create_row("100-200", null, "###") val row6 = create_row("100-200", "(-)", null) - val s = 's.string.at(0) - val p = 'p.string.at(1) - val r = 'r.string.at(2) + val s = "s".attr.string.at(0) + val p = "p".attr.string.at(1) + val r = "r".attr.string.at(2) val expr = RegExpReplace(s, p, r) checkEvaluation(expr, "num-num", row1) @@ -344,9 +344,9 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val row6 = create_row("100-200", null, 1) val row7 = create_row("100-200", "([a-z])", null) - val s = 's.string.at(0) - val p = 'p.string.at(1) - val r = 'r.int.at(2) + val s = "s".attr.string.at(0) + val p = "p".attr.string.at(1) + val r = "r".attr.int.at(2) val expr = RegExpExtract(s, p, r) checkEvaluation(expr, "100", row1) @@ -396,9 +396,9 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val row7 = create_row("100-200,300-400,500-600", null, 1) val row8 = create_row("100-200,300-400,500-600", "([a-z])", null) - val s = 's.string.at(0) - val p = 'p.string.at(1) - val r = 'r.int.at(2) + val s = "s".attr.string.at(0) + val p = "p".attr.string.at(1) + val r = "r".attr.int.at(2) val expr = RegExpExtractAll(s, p, r) checkEvaluation(expr, Seq("100-200", "300-400", "500-600"), row1) @@ -437,8 +437,8 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPLIT") { - val s1 = 'a.string.at(0) - val s2 = 'b.string.at(1) + val s1 = "a".attr.string.at(0) + val s2 = "b".attr.string.at(1) val row1 = create_row("aa2bb3cc", "[1-9]+") val row2 = create_row(null, "[1-9]+") val row3 = create_row("aa2bb3cc", null) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index d25240fec13d..40732a3e6d61 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -139,8 +139,8 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("StringComparison") { val row = create_row("abc", null) - val c1 = 'a.string.at(0) - val c2 = 'a.string.at(1) + val c1 = "a".attr.string.at(0) + val c2 = "a".attr.string.at(1) checkEvaluation(c1 contains "b", true, row) checkEvaluation(c1 contains "x", false, row) @@ -166,7 +166,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("Substring") { val row = create_row("example", "example".toArray.map(_.toByte)) - val s = 'a.string.at(0) + val s = "a".attr.string.at(0) // substring from zero position with less-than-full length checkEvaluation( @@ -240,7 +240,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Substring(s, Literal.create(-1207959552, IntegerType), Literal.create(-1207959552, IntegerType)), "", row) - val s_notNull = 'a.string.notNull.at(0) + val s_notNull = "a".attr.string.notNull.at(0) assert(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable) assert( @@ -302,7 +302,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("ascii for string") { - val a = 'a.string.at(0) + val a = "a".attr.string.at(0) checkEvaluation(Ascii(Literal("efg")), 101, create_row("abdef")) checkEvaluation(Ascii(a), 97, create_row("abdef")) checkEvaluation(Ascii(a), 0, create_row("")) @@ -311,7 +311,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("string for ascii") { - val a = 'a.long.at(0) + val a = "a".attr.long.at(0) checkEvaluation(Chr(Literal(48L)), "0", create_row("abdef")) checkEvaluation(Chr(a), "a", create_row(97L)) checkEvaluation(Chr(a), "a", create_row(97L + 256L)) @@ -324,8 +324,8 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("base64/unbase64 for string") { - val a = 'a.string.at(0) - val b = 'b.binary.at(0) + val a = "a".attr.string.at(0) + val b = "b".attr.binary.at(0) val bytes = Array[Byte](1, 2, 3, 4) checkEvaluation(Base64(Literal(bytes)), "AQIDBA==", create_row("abdef")) @@ -344,8 +344,8 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("encode/decode for string") { - val a = 'a.string.at(0) - val b = 'b.binary.at(0) + val a = "a".attr.string.at(0) + val b = "b".attr.binary.at(0) // scalastyle:off // non ascii characters are not allowed in the code, so we disable the scalastyle here. checkEvaluation( @@ -561,7 +561,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("TRIM") { - val s = 'a.string.at(0) + val s = "a".attr.string.at(0) checkEvaluation(StringTrim(Literal(" aa ")), "aa", create_row(" abdef ")) checkEvaluation(StringTrim("aa", "a"), "", create_row(" abdef ")) checkEvaluation(StringTrim(Literal(" aabbtrimccc"), "ab cd"), "trim", create_row("bdef")) @@ -592,7 +592,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("LTRIM") { - val s = 'a.string.at(0) + val s = "a".attr.string.at(0) checkEvaluation(StringTrimLeft(Literal(" aa ")), "aa ", create_row(" abdef ")) checkEvaluation(StringTrimLeft(Literal("aa"), "a"), "", create_row(" abdef ")) checkEvaluation(StringTrimLeft(Literal("aa "), "a "), "", create_row(" abdef ")) @@ -624,7 +624,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("RTRIM") { - val s = 'a.string.at(0) + val s = "a".attr.string.at(0) checkEvaluation(StringTrimRight(Literal(" aa ")), " aa", create_row(" abdef ")) checkEvaluation(StringTrimRight(Literal("a"), "a"), "", create_row(" abdef ")) checkEvaluation(StringTrimRight(Literal("ab"), "ab"), "", create_row(" abdef ")) @@ -681,9 +681,9 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("INSTR") { - val s1 = 'a.string.at(0) - val s2 = 'b.string.at(1) - val s3 = 'c.string.at(2) + val s1 = "a".attr.string.at(0) + val s2 = "b".attr.string.at(1) + val s3 = "c".attr.string.at(2) val row1 = create_row("aaads", "aa", "zz") checkEvaluation(StringInstr(Literal("aaads"), Literal("aa")), 1, row1) @@ -706,10 +706,10 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("LOCATE") { - val s1 = 'a.string.at(0) - val s2 = 'b.string.at(1) - val s3 = 'c.string.at(2) - val s4 = 'd.int.at(3) + val s1 = "a".attr.string.at(0) + val s2 = "b".attr.string.at(1) + val s3 = "c".attr.string.at(2) + val s4 = "d".attr.int.at(3) val row1 = create_row("aaads", "aa", "zz", 2) val row2 = create_row(null, "aa", "zz", 1) val row3 = create_row("aaads", null, "zz", 1) @@ -733,9 +733,9 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("LPAD/RPAD") { - val s1 = 'a.string.at(0) - val s2 = 'b.int.at(1) - val s3 = 'c.string.at(2) + val s1 = "a".attr.string.at(0) + val s2 = "b".attr.int.at(1) + val s3 = "c".attr.string.at(2) val row1 = create_row("hi", 5, "??") val row2 = create_row("hi", 1, "?") val row3 = create_row(null, 1, "?") @@ -768,8 +768,8 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("REPEAT") { - val s1 = 'a.string.at(0) - val s2 = 'b.int.at(1) + val s1 = "a".attr.string.at(0) + val s2 = "b".attr.int.at(1) val row1 = create_row("hi", 2) val row2 = create_row(null, 1) @@ -783,7 +783,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("REVERSE") { - val s = 'a.string.at(0) + val s = "a".attr.string.at(0) val row1 = create_row("abccc") checkEvaluation(Reverse(Literal("abccc")), "cccba", row1) checkEvaluation(Reverse(s), "cccba", row1) @@ -791,7 +791,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPACE") { - val s1 = 'b.int.at(0) + val s1 = "b".attr.int.at(0) val row1 = create_row(2) val row2 = create_row(null) @@ -803,8 +803,8 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("length for string / binary") { - val a = 'a.string.at(0) - val b = 'b.binary.at(0) + val a = "a".attr.string.at(0) + val b = "b".attr.binary.at(0) val bytes = Array[Byte](1, 2, 3, 1, 2) val string = "abdef" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentileSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentileSuite.scala index 53e8ee9fbe71..1c768bbf74b5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentileSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentileSuite.scala @@ -282,7 +282,7 @@ class ApproximatePercentileSuite extends SparkFunSuite { } test("class ApproximatePercentile, automatically add type casting for parameters") { - val testRelation = LocalRelation('a.int) + val testRelation = LocalRelation("a".attr.int) // accuracy types must be integral, no type casting val accuracyExpressions = Seq( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala index 8984bad479a6..c78de7371ea6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala @@ -35,12 +35,13 @@ class AggregateOptimizeSuite extends AnalysisTest { RemoveRepetitionFromGroupExpressions) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) test("remove literals in grouping expression") { - val query = testRelation.groupBy('a, Literal("1"), Literal(1) + Literal(2))(sum('b)) + val query = + testRelation.groupBy("a".attr, Literal("1"), Literal(1) + Literal(2))(sum("b".attr)) val optimized = Optimize.execute(analyzer.execute(query)) - val correctAnswer = testRelation.groupBy('a)(sum('b)).analyze + val correctAnswer = testRelation.groupBy("a".attr)(sum("b".attr)).analyze comparePlans(optimized, correctAnswer) } @@ -48,26 +49,31 @@ class AggregateOptimizeSuite extends AnalysisTest { test("do not remove all grouping expressions if they are all literals") { withSQLConf(CASE_SENSITIVE.key -> "false", GROUP_BY_ORDINAL.key -> "false") { val analyzer = getAnalyzer - val query = testRelation.groupBy(Literal("1"), Literal(1) + Literal(2))(sum('b)) + val query = testRelation.groupBy(Literal("1"), Literal(1) + Literal(2))(sum("b".attr)) val optimized = Optimize.execute(analyzer.execute(query)) - val correctAnswer = analyzer.execute(testRelation.groupBy(Literal(0))(sum('b))) + val correctAnswer = analyzer.execute(testRelation.groupBy(Literal(0))(sum("b".attr))) comparePlans(optimized, correctAnswer) } } test("Remove aliased literals") { - val query = testRelation.select('a, 'b, Literal(1).as('y)).groupBy('a, 'y)(sum('b)) + val query = testRelation + .select("a".attr, "b".attr, Literal(1).as("y")) + .groupBy("a".attr, "y".attr)(sum("b".attr)) val optimized = Optimize.execute(analyzer.execute(query)) - val correctAnswer = testRelation.select('a, 'b, Literal(1).as('y)).groupBy('a)(sum('b)).analyze + val correctAnswer = testRelation + .select("a".attr, "b".attr, Literal(1).as("y")) + .groupBy("a".attr)(sum("b".attr)).analyze comparePlans(optimized, correctAnswer) } test("remove repetition in grouping expression") { - val query = testRelation.groupBy('a + 1, 'b + 2, Literal(1) + 'A, Literal(2) + 'B)(sum('c)) + val query = testRelation.groupBy("a".attr + 1, + "b".attr + 2, Literal(1) + "A".attr, Literal(2) + "B".attr)(sum("c".attr)) val optimized = Optimize.execute(analyzer.execute(query)) - val correctAnswer = testRelation.groupBy('a + 1, 'b + 2)(sum('c)).analyze + val correctAnswer = testRelation.groupBy("a".attr + 1, "b".attr + 2)(sum("c".attr)).analyze comparePlans(optimized, correctAnswer) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala index c02691848c8f..715f0565c5dd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala @@ -44,12 +44,12 @@ class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper PruneFilters) :: Nil } - val nullableRelation = LocalRelation('a.int.withNullability(true)) - val nonNullableRelation = LocalRelation('a.int.withNullability(false)) + val nullableRelation = LocalRelation("a".attr.int.withNullability(true)) + val nonNullableRelation = LocalRelation("a".attr.int.withNullability(false)) test("Preserve nullable exprs when constraintPropagation is false") { withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "false") { - val a = Symbol("a") + val a = "a".attr for (e <- Seq(a === a, a <= a, a >= a, a < a, a > a)) { val plan = nullableRelation.where(e).analyze val actual = Optimize.execute(plan) @@ -68,7 +68,7 @@ class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper } test("Nullable Simplification Primitive: <=>") { - val plan = nullableRelation.select('a <=> 'a).analyze + val plan = nullableRelation.select("a".attr <=> "a".attr).analyze val actual = Optimize.execute(plan) val correctAnswer = nullableRelation.select(Alias(TrueLiteral, "(a <=> a)")()).analyze comparePlans(actual, correctAnswer) @@ -76,7 +76,8 @@ class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper test("Non-Nullable Simplification Primitive") { val plan = nonNullableRelation - .select('a === 'a, 'a <=> 'a, 'a <= 'a, 'a >= 'a, 'a < 'a, 'a > 'a).analyze + .select("a".attr === "a".attr, "a".attr <=> "a".attr, "a".attr <= "a".attr, + "a".attr >= "a".attr, "a".attr < "a".attr, "a".attr > "a".attr).analyze val actual = Optimize.execute(plan) val correctAnswer = nonNullableRelation .select( @@ -92,8 +93,9 @@ class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper test("Expression Normalization") { val plan = nonNullableRelation.where( - 'a * Literal(100) + Pi() === Pi() + Literal(100) * 'a && - DateAdd(CurrentDate(), 'a + Literal(2)) <= DateAdd(CurrentDate(), Literal(2) + 'a)) + "a".attr * Literal(100) + Pi() === Pi() + Literal(100) * "a".attr && + DateAdd(CurrentDate(), "a".attr + Literal(2)) <= DateAdd(CurrentDate(), + Literal(2) + "a".attr)) .analyze val actual = Optimize.execute(plan) val correctAnswer = nonNullableRelation.analyze @@ -128,7 +130,7 @@ class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper } test("Simplify null and nonnull with filter constraints") { - val a = Symbol("a") + val a = "a".attr Seq(a === a, a <= a, a >= a, a < a, a > a).foreach { condition => val plan = nonNullableRelation.where(condition).analyze val actual = Optimize.execute(plan) @@ -140,7 +142,7 @@ class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper Seq(a === a, a <= a, a >= a).foreach { condition => val plan = nullableRelation.where(condition).analyze val actual = Optimize.execute(plan) - val correctAnswer = nullableRelation.where('a.isNotNull).analyze + val correctAnswer = nullableRelation.where("a".attr.isNotNull).analyze comparePlans(actual, correctAnswer) } @@ -154,13 +156,13 @@ class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper test("Simplify nullable without constraints propagation") { withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "false") { - val a = Symbol("a") + val a = "a".attr Seq(And(a === a, a.isNotNull), And(a <= a, a.isNotNull), And(a >= a, a.isNotNull)).foreach { condition => val plan = nullableRelation.where(condition).analyze val actual = Optimize.execute(plan) - val correctAnswer = nullableRelation.where('a.isNotNull).analyze + val correctAnswer = nullableRelation.where("a".attr.isNotNull).analyze comparePlans(actual, correctAnswer) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala index 04dcf50e0c3c..6c7df6e323cb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala @@ -42,16 +42,17 @@ class BooleanSimplificationSuite extends PlanTest with ExpressionEvalHelper with PruneFilters) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int, 'c.int, 'd.string, - 'e.boolean, 'f.boolean, 'g.boolean, 'h.boolean) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int, "d".attr.string, + "e".attr.boolean, "f".attr.boolean, "g".attr.boolean, "h".attr.boolean) val testRelationWithData = LocalRelation.fromExternalRows( testRelation.output, Seq(Row(1, 2, 3, "abc")) ) - val testNotNullableRelation = LocalRelation('a.int.notNull, 'b.int.notNull, 'c.int.notNull, - 'd.string.notNull, 'e.boolean.notNull, 'f.boolean.notNull, 'g.boolean.notNull, - 'h.boolean.notNull) + val testNotNullableRelation = + LocalRelation("a".attr.int.notNull, "b".attr.int.notNull, "c".attr.int.notNull, + "d".attr.string.notNull, "e".attr.boolean.notNull, "f".attr.boolean.notNull, + "g".attr.boolean.notNull, "h".attr.boolean.notNull) val testNotNullableRelationWithData = LocalRelation.fromExternalRows( testNotNullableRelation.output, Seq(Row(1, 2, 3, "abc")) @@ -86,105 +87,129 @@ class BooleanSimplificationSuite extends PlanTest with ExpressionEvalHelper with } test("a && a => a") { - checkCondition(Literal(1) < 'a && Literal(1) < 'a, Literal(1) < 'a) - checkCondition(Literal(1) < 'a && Literal(1) < 'a && Literal(1) < 'a, Literal(1) < 'a) + checkCondition(Literal(1) < "a".attr && Literal(1) < "a".attr, Literal(1) < "a".attr) + checkCondition(Literal(1) < "a".attr && Literal(1) < "a".attr && + Literal(1) < "a".attr, Literal(1) < "a".attr) } test("a || a => a") { - checkCondition(Literal(1) < 'a || Literal(1) < 'a, Literal(1) < 'a) - checkCondition(Literal(1) < 'a || Literal(1) < 'a || Literal(1) < 'a, Literal(1) < 'a) + checkCondition(Literal(1) < "a".attr || Literal(1) < "a".attr, + Literal(1) < "a".attr) + checkCondition(Literal(1) < "a".attr || Literal(1) < "a".attr || + Literal(1) < "a".attr, Literal(1) < "a".attr) } test("(a && b && c && ...) || (a && b && d && ...) || (a && b && e && ...) ...") { - checkCondition('b > 3 || 'c > 5, 'b > 3 || 'c > 5) + checkCondition("b".attr > 3 || "c".attr > 5, "b".attr > 3 || "c".attr > 5) - checkCondition(('a < 2 && 'a > 3 && 'b > 5) || 'a < 2, 'a < 2) + checkCondition(("a".attr < 2 && "a".attr > 3 && "b".attr > 5) || "a".attr < 2, "a".attr < 2) - checkCondition('a < 2 || ('a < 2 && 'a > 3 && 'b > 5), 'a < 2) + checkCondition("a".attr < 2 || ("a".attr < 2 && "a".attr > 3 && "b".attr > 5), "a".attr < 2) - val input = ('a === 'b && 'b > 3 && 'c > 2) || - ('a === 'b && 'c < 1 && 'a === 5) || - ('a === 'b && 'b < 5 && 'a > 1) + val input = ("a".attr === "b".attr && "b".attr > 3 && "c".attr > 2) || + ("a".attr === "b".attr && "c".attr < 1 && "a".attr === 5) || + ("a".attr === "b".attr && "b".attr < 5 && "a".attr > 1) - val expected = 'a === 'b && ( - ('b > 3 && 'c > 2) || ('c < 1 && 'a === 5) || ('b < 5 && 'a > 1)) + val expected = "a".attr === "b".attr && ( + ("b".attr > 3 && "c".attr > 2) || ("c".attr < 1 && "a".attr === 5) || + ("b".attr < 5 && "a".attr > 1)) checkCondition(input, expected) } test("(a || b || c || ...) && (a || b || d || ...) && (a || b || e || ...) ...") { - checkCondition('b > 3 && 'c > 5, 'b > 3 && 'c > 5) + checkCondition("b".attr > 3 && "c".attr > 5, "b".attr > 3 && "c".attr > 5) - checkCondition(('a < 2 || 'a > 3 || 'b > 5) && 'a < 2, 'a < 2) + checkCondition(("a".attr < 2 || "a".attr > 3 || "b".attr > 5) && "a".attr < 2, "a".attr < 2) - checkCondition('a < 2 && ('a < 2 || 'a > 3 || 'b > 5), 'a < 2) + checkCondition("a".attr < 2 && ("a".attr < 2 || "a".attr > 3 || "b".attr > 5), "a".attr < 2) - checkCondition(('a < 2 || 'b > 3) && ('a < 2 || 'c > 5), 'a < 2 || ('b > 3 && 'c > 5)) + checkCondition(("a".attr < 2 || "b".attr > 3) && ("a".attr < 2 || "c".attr > 5), "a".attr < 2 + || ("b".attr > 3 && "c".attr > 5)) checkCondition( - ('a === 'b || 'b > 3) && ('a === 'b || 'a > 3) && ('a === 'b || 'a < 5), - 'a === 'b || 'b > 3 && 'a > 3 && 'a < 5) + ("a".attr === "b".attr || "b".attr > 3) && ("a".attr === "b".attr || "a".attr > 3) && + ("a".attr === "b".attr || "a".attr < 5), + "a".attr === "b".attr || "b".attr > 3 && "a".attr > 3 && "a".attr < 5) } test("e && (!e || f) - not nullable") { - checkConditionInNotNullableRelation('e && (!'e || 'f ), 'e && 'f) + checkConditionInNotNullableRelation("e".attr && (!"e".attr || "f".attr ), "e".attr && "f".attr) - checkConditionInNotNullableRelation('e && ('f || !'e ), 'e && 'f) + checkConditionInNotNullableRelation("e".attr && ("f".attr || !"e".attr ), "e".attr && "f".attr) - checkConditionInNotNullableRelation((!'e || 'f ) && 'e, 'f && 'e) + checkConditionInNotNullableRelation((!"e".attr || "f".attr ) && "e".attr, "f".attr && "e".attr) - checkConditionInNotNullableRelation(('f || !'e ) && 'e, 'f && 'e) + checkConditionInNotNullableRelation(("f".attr || !"e".attr ) && "e".attr, "f".attr && "e".attr) } test("e && (!e || f) - nullable") { - Seq ('e && (!'e || 'f ), - 'e && ('f || !'e ), - (!'e || 'f ) && 'e, - ('f || !'e ) && 'e, - 'e || (!'e && 'f), - 'e || ('f && !'e), - ('e && 'f) || !'e, - ('f && 'e) || !'e).foreach { expr => + Seq ("e".attr && (!"e".attr || "f".attr ), + "e".attr && ("f".attr || !"e".attr ), + (!"e".attr || "f".attr ) && "e".attr, + ("f".attr || !"e".attr ) && "e".attr, + "e".attr || (!"e".attr && "f".attr), + "e".attr || ("f".attr && !"e".attr), + ("e".attr && "f".attr) || !"e".attr, + ("f".attr && "e".attr) || !"e".attr).foreach { expr => checkCondition(expr, expr) } } test("a < 1 && (!(a < 1) || f) - not nullable") { - checkConditionInNotNullableRelation('a < 1 && (!('a < 1) || 'f), ('a < 1) && 'f) - checkConditionInNotNullableRelation('a < 1 && ('f || !('a < 1)), ('a < 1) && 'f) - - checkConditionInNotNullableRelation('a <= 1 && (!('a <= 1) || 'f), ('a <= 1) && 'f) - checkConditionInNotNullableRelation('a <= 1 && ('f || !('a <= 1)), ('a <= 1) && 'f) - - checkConditionInNotNullableRelation('a > 1 && (!('a > 1) || 'f), ('a > 1) && 'f) - checkConditionInNotNullableRelation('a > 1 && ('f || !('a > 1)), ('a > 1) && 'f) - - checkConditionInNotNullableRelation('a >= 1 && (!('a >= 1) || 'f), ('a >= 1) && 'f) - checkConditionInNotNullableRelation('a >= 1 && ('f || !('a >= 1)), ('a >= 1) && 'f) + checkConditionInNotNullableRelation("a".attr < 1 && (!("a".attr < 1) || "f".attr), + ("a".attr < 1) && "f".attr) + checkConditionInNotNullableRelation("a".attr < 1 && ("f".attr || !("a".attr < 1)), + ("a".attr < 1) && "f".attr) + + checkConditionInNotNullableRelation("a".attr <= 1 && (!("a".attr <= 1) || "f".attr), + ("a".attr <= 1) && "f".attr) + checkConditionInNotNullableRelation("a".attr <= 1 && ("f".attr || !("a".attr <= 1)), + ("a".attr <= 1) && "f".attr) + + checkConditionInNotNullableRelation("a".attr > 1 && (!("a".attr > 1) || "f".attr), + ("a".attr > 1) && "f".attr) + checkConditionInNotNullableRelation("a".attr > 1 && ("f".attr || !("a".attr > 1)), + ("a".attr > 1) && "f".attr) + + checkConditionInNotNullableRelation("a".attr >= 1 && (!("a".attr >= 1) || "f".attr), + ("a".attr >= 1) && "f".attr) + checkConditionInNotNullableRelation("a".attr >= 1 && ("f".attr || !("a".attr >= 1)), + ("a".attr >= 1) && "f".attr) } test("a < 1 && ((a >= 1) || f) - not nullable") { - checkConditionInNotNullableRelation('a < 1 && ('a >= 1 || 'f ), ('a < 1) && 'f) - checkConditionInNotNullableRelation('a < 1 && ('f || 'a >= 1), ('a < 1) && 'f) - - checkConditionInNotNullableRelation('a <= 1 && ('a > 1 || 'f ), ('a <= 1) && 'f) - checkConditionInNotNullableRelation('a <= 1 && ('f || 'a > 1), ('a <= 1) && 'f) - - checkConditionInNotNullableRelation('a > 1 && (('a <= 1) || 'f), ('a > 1) && 'f) - checkConditionInNotNullableRelation('a > 1 && ('f || ('a <= 1)), ('a > 1) && 'f) - - checkConditionInNotNullableRelation('a >= 1 && (('a < 1) || 'f), ('a >= 1) && 'f) - checkConditionInNotNullableRelation('a >= 1 && ('f || ('a < 1)), ('a >= 1) && 'f) + checkConditionInNotNullableRelation("a".attr < 1 && ("a".attr >= 1 || "f".attr ), + ("a".attr < 1) && "f".attr) + checkConditionInNotNullableRelation("a".attr < 1 && ("f".attr || "a".attr >= 1), + ("a".attr < 1) && "f".attr) + + checkConditionInNotNullableRelation("a".attr <= 1 && ("a".attr > 1 || "f".attr ), + ("a".attr <= 1) && "f".attr) + checkConditionInNotNullableRelation("a".attr <= 1 && ("f".attr || "a".attr > 1), + ("a".attr <= 1) && "f".attr) + + checkConditionInNotNullableRelation("a".attr > 1 && (("a".attr <= 1) || "f".attr), + ("a".attr > 1) && "f".attr) + checkConditionInNotNullableRelation("a".attr > 1 && ("f".attr || ("a".attr <= 1)), + ("a".attr > 1) && "f".attr) + + checkConditionInNotNullableRelation("a".attr >= 1 && (("a".attr < 1) || "f".attr), + ("a".attr >= 1) && "f".attr) + checkConditionInNotNullableRelation("a".attr >= 1 && ("f".attr || ("a".attr < 1)), + ("a".attr >= 1) && "f".attr) } test("DeMorgan's law") { - checkCondition(!('e && 'f), !'e || !'f) + checkCondition(!("e".attr && "f".attr), !"e".attr || !"f".attr) - checkCondition(!('e || 'f), !'e && !'f) + checkCondition(!("e".attr || "f".attr), !"e".attr && !"f".attr) - checkCondition(!(('e && 'f) || ('g && 'h)), (!'e || !'f) && (!'g || !'h)) + checkCondition(!(("e".attr && "f".attr) || ("g".attr && "h".attr)), (!"e".attr || !"f".attr) && + (!"g".attr || !"h".attr)) - checkCondition(!(('e || 'f) && ('g || 'h)), (!'e && !'f) || (!'g && !'h)) + checkCondition(!(("e".attr || "f".attr) && ("g".attr || "h".attr)), (!"e".attr && !"f".attr) || + (!"g".attr && !"h".attr)) } private val analyzer = new Analyzer( @@ -192,53 +217,57 @@ class BooleanSimplificationSuite extends PlanTest with ExpressionEvalHelper with test("(a && b) || (a && c) => a && (b || c) when case insensitive") { val plan = analyzer.execute( - testRelation.where(('a > 2 && 'b > 3) || ('A > 2 && 'b < 5))) + testRelation.where(("a".attr > 2 && "b".attr > 3) || ("A".attr > 2 && "b".attr < 5))) val actual = Optimize.execute(plan) val expected = analyzer.execute( - testRelation.where('a > 2 && ('b > 3 || 'b < 5))) + testRelation.where("a".attr > 2 && ("b".attr > 3 || "b".attr < 5))) comparePlans(actual, expected) } test("(a || b) && (a || c) => a || (b && c) when case insensitive") { val plan = analyzer.execute( - testRelation.where(('a > 2 || 'b > 3) && ('A > 2 || 'b < 5))) + testRelation.where(("a".attr > 2 || "b".attr > 3) && ("A".attr > 2 || "b".attr < 5))) val actual = Optimize.execute(plan) val expected = analyzer.execute( - testRelation.where('a > 2 || ('b > 3 && 'b < 5))) + testRelation.where("a".attr > 2 || ("b".attr > 3 && "b".attr < 5))) comparePlans(actual, expected) } test("Complementation Laws") { - checkConditionInNotNullableRelation('e && !'e, testNotNullableRelation) - checkConditionInNotNullableRelation(!'e && 'e, testNotNullableRelation) + checkConditionInNotNullableRelation("e".attr && !"e".attr, testNotNullableRelation) + checkConditionInNotNullableRelation(!"e".attr && "e".attr, testNotNullableRelation) - checkConditionInNotNullableRelation('e || !'e, testNotNullableRelationWithData) - checkConditionInNotNullableRelation(!'e || 'e, testNotNullableRelationWithData) + checkConditionInNotNullableRelation("e".attr || !"e".attr, testNotNullableRelationWithData) + checkConditionInNotNullableRelation(!"e".attr || "e".attr, testNotNullableRelationWithData) } test("Complementation Laws - null handling") { - checkCondition('e && !'e, - testRelationWithData.where(And(Literal(null, BooleanType), 'e.isNull)).analyze) - checkCondition(!'e && 'e, - testRelationWithData.where(And(Literal(null, BooleanType), 'e.isNull)).analyze) - - checkCondition('e || !'e, - testRelationWithData.where(Or('e.isNotNull, Literal(null, BooleanType))).analyze) - checkCondition(!'e || 'e, - testRelationWithData.where(Or('e.isNotNull, Literal(null, BooleanType))).analyze) + checkCondition("e".attr && !"e".attr, + testRelationWithData.where(And(Literal(null, BooleanType), "e".attr.isNull)).analyze) + checkCondition(!"e".attr && "e".attr, + testRelationWithData.where(And(Literal(null, BooleanType), "e".attr.isNull)).analyze) + + checkCondition("e".attr || !"e".attr, + testRelationWithData.where(Or("e".attr.isNotNull, Literal(null, BooleanType))).analyze) + checkCondition(!"e".attr || "e".attr, + testRelationWithData.where(Or("e".attr.isNotNull, Literal(null, BooleanType))).analyze) } test("Complementation Laws - negative case") { - checkCondition('e && !'f, testRelationWithData.where('e && !'f).analyze) - checkCondition(!'f && 'e, testRelationWithData.where(!'f && 'e).analyze) - - checkCondition('e || !'f, testRelationWithData.where('e || !'f).analyze) - checkCondition(!'f || 'e, testRelationWithData.where(!'f || 'e).analyze) + checkCondition("e".attr && !"f".attr, + testRelationWithData.where("e".attr && !"f".attr).analyze) + checkCondition(!"f".attr && "e".attr, + testRelationWithData.where(!"f".attr && "e".attr).analyze) + + checkCondition("e".attr || !"f".attr, + testRelationWithData.where("e".attr || !"f".attr).analyze) + checkCondition(!"f".attr || "e".attr, + testRelationWithData.where(!"f".attr || "e".attr).analyze) } test("simplify NOT(IsNull(x)) and NOT(IsNotNull(x))") { - checkCondition(Not(IsNotNull('b)), IsNull('b)) - checkCondition(Not(IsNull('b)), IsNotNull('b)) + checkCondition(Not(IsNotNull("b".attr)), IsNull("b".attr)) + checkCondition(Not(IsNull("b".attr)), IsNotNull("b".attr)) } protected def assertEquivalent(e1: Expression, e2: Expression): Unit = { @@ -249,8 +278,8 @@ class BooleanSimplificationSuite extends PlanTest with ExpressionEvalHelper with test("filter reduction - positive cases") { val fields = Seq( - 'col1NotNULL.boolean.notNull, - 'col2NotNULL.boolean.notNull + "col1NotNULL".attr.boolean.notNull, + "col2NotNULL".attr.boolean.notNull ) val Seq(col1NotNULL, col2NotNULL) = fields.zipWithIndex.map { case (f, i) => f.at(i) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CheckCartesianProductsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CheckCartesianProductsSuite.scala index dea2b36ecc84..69f743188819 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CheckCartesianProductsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CheckCartesianProductsSuite.scala @@ -34,11 +34,11 @@ class CheckCartesianProductsSuite extends PlanTest { val batches = Batch("Check Cartesian Products", Once, CheckCartesianProducts) :: Nil } - val testRelation1 = LocalRelation('a.int, 'b.int) - val testRelation2 = LocalRelation('c.int, 'd.int) + val testRelation1 = LocalRelation("a".attr.int, "b".attr.int) + val testRelation2 = LocalRelation("c".attr.int, "d".attr.int) val joinTypesWithRequiredCondition = Seq(Inner, LeftOuter, RightOuter, FullOuter) - val joinTypesWithoutRequiredCondition = Seq(LeftSemi, LeftAnti, ExistenceJoin('exists)) + val joinTypesWithoutRequiredCondition = Seq(LeftSemi, LeftAnti, ExistenceJoin("exists".attr)) test("CheckCartesianProducts doesn't throw an exception if cross joins are enabled)") { withSQLConf(CROSS_JOINS_ENABLED.key -> "true") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala index 42bcd13ee378..d5a90b9033e1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala @@ -33,37 +33,38 @@ class CollapseProjectSuite extends PlanTest { Batch("CollapseProject", Once, CollapseProject) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int) test("collapse two deterministic, independent projects into one") { val query = testRelation - .select(('a + 1).as('a_plus_1), 'b) - .select('a_plus_1, ('b + 1).as('b_plus_1)) + .select(("a".attr + 1).as("a_plus_1"), "b".attr) + .select("a_plus_1".attr, ("b".attr + 1).as("b_plus_1")) val optimized = Optimize.execute(query.analyze) - val correctAnswer = testRelation.select(('a + 1).as('a_plus_1), ('b + 1).as('b_plus_1)).analyze + val correctAnswer = + testRelation.select(("a".attr + 1).as("a_plus_1"), ("b".attr + 1).as("b_plus_1")).analyze comparePlans(optimized, correctAnswer) } test("collapse two deterministic, dependent projects into one") { val query = testRelation - .select(('a + 1).as('a_plus_1), 'b) - .select(('a_plus_1 + 1).as('a_plus_2), 'b) + .select(("a".attr + 1).as("a_plus_1"), "b".attr) + .select(("a_plus_1".attr + 1).as("a_plus_2"), "b".attr) val optimized = Optimize.execute(query.analyze) val correctAnswer = testRelation.select( - (('a + 1).as('a_plus_1) + 1).as('a_plus_2), - 'b).analyze + (("a".attr + 1).as("a_plus_1") + 1).as("a_plus_2"), + "b".attr).analyze comparePlans(optimized, correctAnswer) } test("do not collapse nondeterministic projects") { val query = testRelation - .select(Rand(10).as('rand)) - .select(('rand + 1).as('rand1), ('rand + 2).as('rand2)) + .select(Rand(10).as("rand")) + .select(("rand".attr + 1).as("rand1"), ("rand".attr + 2).as("rand2")) val optimized = Optimize.execute(query.analyze) val correctAnswer = query.analyze @@ -73,47 +74,48 @@ class CollapseProjectSuite extends PlanTest { test("collapse two nondeterministic, independent projects into one") { val query = testRelation - .select(Rand(10).as('rand)) - .select(Rand(20).as('rand2)) + .select(Rand(10).as("rand")) + .select(Rand(20).as("rand2")) val optimized = Optimize.execute(query.analyze) val correctAnswer = testRelation - .select(Rand(20).as('rand2)).analyze + .select(Rand(20).as("rand2")).analyze comparePlans(optimized, correctAnswer) } test("collapse one nondeterministic, one deterministic, independent projects into one") { val query = testRelation - .select(Rand(10).as('rand), 'a) - .select(('a + 1).as('a_plus_1)) + .select(Rand(10).as("rand"), "a".attr) + .select(("a".attr + 1).as("a_plus_1")) val optimized = Optimize.execute(query.analyze) val correctAnswer = testRelation - .select(('a + 1).as('a_plus_1)).analyze + .select(("a".attr + 1).as("a_plus_1")).analyze comparePlans(optimized, correctAnswer) } test("collapse project into aggregate") { val query = testRelation - .groupBy('a, 'b)(('a + 1).as('a_plus_1), 'b) - .select('a_plus_1, ('b + 1).as('b_plus_1)) + .groupBy("a".attr, "b".attr)(("a".attr + 1).as("a_plus_1"), "b".attr) + .select("a_plus_1".attr, ("b".attr + 1).as("b_plus_1")) val optimized = Optimize.execute(query.analyze) val correctAnswer = testRelation - .groupBy('a, 'b)(('a + 1).as('a_plus_1), ('b + 1).as('b_plus_1)).analyze + .groupBy("a".attr, "b".attr)(("a".attr + 1).as("a_plus_1"), ("b".attr + 1).as("b_plus_1")) + .analyze comparePlans(optimized, correctAnswer) } test("do not collapse common nondeterministic project and aggregate") { val query = testRelation - .groupBy('a)('a, Rand(10).as('rand)) - .select(('rand + 1).as('rand1), ('rand + 2).as('rand2)) + .groupBy("a".attr)("a".attr, Rand(10).as("rand")) + .select(("rand".attr + 1).as("rand1"), ("rand".attr + 2).as("rand2")) val optimized = Optimize.execute(query.analyze) val correctAnswer = query.analyze @@ -128,8 +130,8 @@ class CollapseProjectSuite extends PlanTest { val metadata = new MetadataBuilder().putLong("key", 1).build() val analyzed = - Project(Seq(Alias('a_with_metadata, "b")()), - Project(Seq(Alias('a, "a_with_metadata")(explicitMetadata = Some(metadata))), + Project(Seq(Alias("a_with_metadata".attr, "b")()), + Project(Seq(Alias("a".attr, "a_with_metadata")(explicitMetadata = Some(metadata))), testRelation.logicalPlan)).analyze require(hasMetadata(analyzed)) @@ -140,34 +142,35 @@ class CollapseProjectSuite extends PlanTest { } test("collapse redundant alias through limit") { - val relation = LocalRelation('a.int, 'b.int) - val query = relation.select('a as 'b).limit(1).select('b as 'c).analyze + val relation = LocalRelation("a".attr.int, "b".attr.int) + val query = relation.select("a".attr as "b").limit(1).select("b".attr as "c").analyze val optimized = Optimize.execute(query) - val expected = relation.select('a as 'c).limit(1).analyze + val expected = relation.select("a".attr as "c").limit(1).analyze comparePlans(optimized, expected) } test("collapse redundant alias through local limit") { - val relation = LocalRelation('a.int, 'b.int) - val query = LocalLimit(1, relation.select('a as 'b)).select('b as 'c).analyze + val relation = LocalRelation("a".attr.int, "b".attr.int) + val query = LocalLimit(1, relation.select("a".attr as "b")).select("b".attr as "c").analyze val optimized = Optimize.execute(query) - val expected = LocalLimit(1, relation.select('a as 'c)).analyze + val expected = LocalLimit(1, relation.select("a".attr as "c")).analyze comparePlans(optimized, expected) } test("collapse redundant alias through repartition") { - val relation = LocalRelation('a.int, 'b.int) - val query = relation.select('a as 'b).repartition(1).select('b as 'c).analyze + val relation = LocalRelation("a".attr.int, "b".attr.int) + val query = relation.select("a".attr as "b").repartition(1).select("b".attr as "c").analyze val optimized = Optimize.execute(query) - val expected = relation.select('a as 'c).repartition(1).analyze + val expected = relation.select("a".attr as "c").repartition(1).analyze comparePlans(optimized, expected) } test("collapse redundant alias through sample") { - val relation = LocalRelation('a.int, 'b.int) - val query = Sample(0.0, 0.6, false, 11L, relation.select('a as 'b)).select('b as 'c).analyze + val relation = LocalRelation("a".attr.int, "b".attr.int) + val query = Sample(0.0, 0.6, false, 11L, relation.select("a".attr as "b")) + .select("b".attr as "c").analyze val optimized = Optimize.execute(query) - val expected = Sample(0.0, 0.6, false, 11L, relation.select('a as 'c)).analyze + val expected = Sample(0.0, 0.6, false, 11L, relation.select("a".attr as "c")).analyze comparePlans(optimized, expected) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseRepartitionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseRepartitionSuite.scala index 8cc8decd65de..4fb2f456eb70 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseRepartitionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseRepartitionSuite.scala @@ -30,7 +30,7 @@ class CollapseRepartitionSuite extends PlanTest { CollapseRepartition) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int) test("collapse two adjacent coalesces into one") { @@ -110,14 +110,14 @@ class CollapseRepartitionSuite extends PlanTest { // Always respects the top distribute and removes useless repartition val query1 = testRelation .repartition(10) - .distribute('a)(20) + .distribute("a".attr)(20) val query2 = testRelation .repartition(30) - .distribute('a)(20) + .distribute("a".attr)(20) val optimized1 = Optimize.execute(query1.analyze) val optimized2 = Optimize.execute(query2.analyze) - val correctAnswer = testRelation.distribute('a)(20).analyze + val correctAnswer = testRelation.distribute("a".attr)(20).analyze comparePlans(optimized1, correctAnswer) comparePlans(optimized2, correctAnswer) @@ -127,14 +127,14 @@ class CollapseRepartitionSuite extends PlanTest { // Always respects the top distribute and removes useless coalesce below repartition val query1 = testRelation .coalesce(10) - .distribute('a)(20) + .distribute("a".attr)(20) val query2 = testRelation .coalesce(30) - .distribute('a)(20) + .distribute("a".attr)(20) val optimized1 = Optimize.execute(query1.analyze) val optimized2 = Optimize.execute(query2.analyze) - val correctAnswer = testRelation.distribute('a)(20).analyze + val correctAnswer = testRelation.distribute("a".attr)(20).analyze comparePlans(optimized1, correctAnswer) comparePlans(optimized2, correctAnswer) @@ -143,10 +143,10 @@ class CollapseRepartitionSuite extends PlanTest { test("repartition above distribute") { // Always respects the top repartition and removes useless distribute below repartition val query1 = testRelation - .distribute('a)(10) + .distribute("a".attr)(10) .repartition(20) val query2 = testRelation - .distribute('a)(30) + .distribute("a".attr)(30) .repartition(20) val optimized1 = Optimize.execute(query1.analyze) @@ -160,17 +160,17 @@ class CollapseRepartitionSuite extends PlanTest { test("coalesce above distribute") { // Remove useless coalesce above distribute val query1 = testRelation - .distribute('a)(10) + .distribute("a".attr)(10) .coalesce(20) val optimized1 = Optimize.execute(query1.analyze) - val correctAnswer1 = testRelation.distribute('a)(10).analyze + val correctAnswer1 = testRelation.distribute("a".attr)(10).analyze comparePlans(optimized1, correctAnswer1) // No change in this case val query2 = testRelation - .distribute('a)(30) + .distribute("a".attr)(30) .coalesce(20) val optimized2 = Optimize.execute(query2.analyze) @@ -182,15 +182,15 @@ class CollapseRepartitionSuite extends PlanTest { test("collapse two adjacent distributes into one") { // Always respects the top distribute val query1 = testRelation - .distribute('b)(10) - .distribute('a)(20) + .distribute("b".attr)(10) + .distribute("a".attr)(20) val query2 = testRelation - .distribute('b)(30) - .distribute('a)(20) + .distribute("b".attr)(30) + .distribute("a".attr)(20) val optimized1 = Optimize.execute(query1.analyze) val optimized2 = Optimize.execute(query2.analyze) - val correctAnswer = testRelation.distribute('a)(20).analyze + val correctAnswer = testRelation.distribute("a".attr)(20).analyze comparePlans(optimized1, correctAnswer) comparePlans(optimized2, correctAnswer) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseWindowSuite.scala index 3b3b4907eea8..719fc9bf7a51 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseWindowSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseWindowSuite.scala @@ -30,7 +30,7 @@ class CollapseWindowSuite extends PlanTest { CollapseWindow) :: Nil } - val testRelation = LocalRelation('a.double, 'b.double, 'c.string) + val testRelation = LocalRelation("a".attr.double, "b".attr.double, "c".attr.string) val a = testRelation.output(0) val b = testRelation.output(1) val c = testRelation.output(2) @@ -41,28 +41,28 @@ class CollapseWindowSuite extends PlanTest { test("collapse two adjacent windows with the same partition/order") { val query = testRelation - .window(Seq(min(a).as('min_a)), partitionSpec1, orderSpec1) - .window(Seq(max(a).as('max_a)), partitionSpec1, orderSpec1) - .window(Seq(sum(b).as('sum_b)), partitionSpec1, orderSpec1) - .window(Seq(avg(b).as('avg_b)), partitionSpec1, orderSpec1) + .window(Seq(min(a).as("min_a")), partitionSpec1, orderSpec1) + .window(Seq(max(a).as("max_a")), partitionSpec1, orderSpec1) + .window(Seq(sum(b).as("sum_b")), partitionSpec1, orderSpec1) + .window(Seq(avg(b).as("avg_b")), partitionSpec1, orderSpec1) val analyzed = query.analyze val optimized = Optimize.execute(analyzed) assert(analyzed.output === optimized.output) val correctAnswer = testRelation.window(Seq( - min(a).as('min_a), - max(a).as('max_a), - sum(b).as('sum_b), - avg(b).as('avg_b)), partitionSpec1, orderSpec1) + min(a).as("min_a"), + max(a).as("max_a"), + sum(b).as("sum_b"), + avg(b).as("avg_b")), partitionSpec1, orderSpec1) comparePlans(optimized, correctAnswer) } test("Don't collapse adjacent windows with different partitions or orders") { val query1 = testRelation - .window(Seq(min(a).as('min_a)), partitionSpec1, orderSpec1) - .window(Seq(max(a).as('max_a)), partitionSpec1, orderSpec2) + .window(Seq(min(a).as("min_a")), partitionSpec1, orderSpec1) + .window(Seq(max(a).as("max_a")), partitionSpec1, orderSpec2) val optimized1 = Optimize.execute(query1.analyze) val correctAnswer1 = query1.analyze @@ -70,8 +70,8 @@ class CollapseWindowSuite extends PlanTest { comparePlans(optimized1, correctAnswer1) val query2 = testRelation - .window(Seq(min(a).as('min_a)), partitionSpec1, orderSpec1) - .window(Seq(max(a).as('max_a)), partitionSpec2, orderSpec1) + .window(Seq(min(a).as("min_a")), partitionSpec1, orderSpec1) + .window(Seq(max(a).as("max_a")), partitionSpec2, orderSpec1) val optimized2 = Optimize.execute(query2.analyze) val correctAnswer2 = query2.analyze @@ -81,8 +81,8 @@ class CollapseWindowSuite extends PlanTest { test("Don't collapse adjacent windows with dependent columns") { val query = testRelation - .window(Seq(sum(a).as('sum_a)), partitionSpec1, orderSpec1) - .window(Seq(max('sum_a).as('max_sum_a)), partitionSpec1, orderSpec1) + .window(Seq(sum(a).as("sum_a")), partitionSpec1, orderSpec1) + .window(Seq(max("sum_a".attr).as("max_sum_a")), partitionSpec1, orderSpec1) .analyze val expected = query.analyze @@ -93,7 +93,7 @@ class CollapseWindowSuite extends PlanTest { test("Skip windows with empty window expressions") { val query = testRelation .window(Seq(), partitionSpec1, orderSpec1) - .window(Seq(sum(a).as('sum_a)), partitionSpec1, orderSpec1) + .window(Seq(sum(a).as("sum_a")), partitionSpec1, orderSpec1) val optimized = Optimize.execute(query.analyze) val correctAnswer = query.analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala index bfa415afeab9..80b4f75dddf9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala @@ -41,61 +41,61 @@ class ColumnPruningSuite extends PlanTest { } test("Column pruning for Generate when Generate.unrequiredChildIndex = child.output") { - val input = LocalRelation('a.int, 'b.int, 'c.array(StringType)) + val input = LocalRelation("a".attr.int, "b".attr.int, "c".attr.array(StringType)) val query = input - .generate(Explode('c), outputNames = "explode" :: Nil) - .select('c, 'explode) + .generate(Explode("c".attr), outputNames = "explode" :: Nil) + .select("c".attr, "explode".attr) .analyze val optimized = Optimize.execute(query) val correctAnswer = input - .select('c) - .generate(Explode('c), outputNames = "explode" :: Nil) + .select("c".attr) + .generate(Explode("c".attr), outputNames = "explode" :: Nil) .analyze comparePlans(optimized, correctAnswer) } test("Fill Generate.unrequiredChildIndex if possible") { - val input = LocalRelation('b.array(StringType)) + val input = LocalRelation("b".attr.array(StringType)) val query = input - .generate(Explode('b), outputNames = "explode" :: Nil) - .select(('explode + 1).as("result")) + .generate(Explode("b".attr), outputNames = "explode" :: Nil) + .select(("explode".attr + 1).as("result")) .analyze val optimized = Optimize.execute(query) val correctAnswer = input - .generate(Explode('b), unrequiredChildIndex = input.output.zipWithIndex.map(_._2), + .generate(Explode("b".attr), unrequiredChildIndex = input.output.zipWithIndex.map(_._2), outputNames = "explode" :: Nil) - .select(('explode + 1).as("result")) + .select(("explode".attr + 1).as("result")) .analyze comparePlans(optimized, correctAnswer) } test("Another fill Generate.unrequiredChildIndex if possible") { - val input = LocalRelation('a.int, 'b.int, 'c1.string, 'c2.string) + val input = LocalRelation("a".attr.int, "b".attr.int, "c1".attr.string, "c2".attr.string) val query = input - .generate(Explode(CreateArray(Seq('c1, 'c2))), outputNames = "explode" :: Nil) - .select('a, 'c1, 'explode) + .generate(Explode(CreateArray(Seq("c1".attr, "c2".attr))), outputNames = "explode" :: Nil) + .select("a".attr, "c1".attr, "explode".attr) .analyze val optimized = Optimize.execute(query) val correctAnswer = input - .select('a, 'c1, 'c2) - .generate(Explode(CreateArray(Seq('c1, 'c2))), + .select("a".attr, "c1".attr, "c2".attr) + .generate(Explode(CreateArray(Seq("c1".attr, "c2".attr))), unrequiredChildIndex = Seq(2), outputNames = "explode" :: Nil) .analyze @@ -113,10 +113,10 @@ class ColumnPruningSuite extends PlanTest { withSQLConf(SQLConf.NESTED_PRUNING_ON_EXPRESSIONS.key -> "true") { val structType = StructType.fromDDL("d double, e array, f double, g double, " + "h array>") - val input = LocalRelation('a.int, 'b.int, 'c.struct(structType)) + val input = LocalRelation("a".attr.int, "b".attr.int, "c".attr.struct(structType)) val generatorOutputs = generatorOutputNames.map(UnresolvedAttribute(_)) - val selectedExprs = Seq(UnresolvedAttribute("a"), 'c.getField("d")) ++ + val selectedExprs = Seq(UnresolvedAttribute("a"), "c".attr.getField("d")) ++ generatorOutputs val query = @@ -147,82 +147,83 @@ class ColumnPruningSuite extends PlanTest { } runTest( - Explode('c.getField("e")), + Explode("c".attr.getField("e")), aliases => Explode($"${aliases(1)}".as("c.e")), - aliases => Seq('c.getField("d").as(aliases(0)), 'c.getField("e").as(aliases(1))), + aliases => Seq("c".attr.getField("d").as(aliases(0)), "c".attr.getField("e").as(aliases(1))), Seq(2), Seq("explode") ) - runTest(Stack(2 :: 'c.getField("f") :: 'c.getField("g") :: Nil), + runTest(Stack(2 :: "c".attr.getField("f") :: "c".attr.getField("g") :: Nil), aliases => Stack(2 :: $"${aliases(1)}".as("c.f") :: $"${aliases(2)}".as("c.g") :: Nil), aliases => Seq( - 'c.getField("d").as(aliases(0)), - 'c.getField("f").as(aliases(1)), - 'c.getField("g").as(aliases(2))), + "c".attr.getField("d").as(aliases(0)), + "c".attr.getField("f").as(aliases(1)), + "c".attr.getField("g").as(aliases(2))), Seq(2, 3), Seq("stack") ) runTest( - PosExplode('c.getField("e")), + PosExplode("c".attr.getField("e")), aliases => PosExplode($"${aliases(1)}".as("c.e")), - aliases => Seq('c.getField("d").as(aliases(0)), 'c.getField("e").as(aliases(1))), + aliases => Seq("c".attr.getField("d").as(aliases(0)), "c".attr.getField("e").as(aliases(1))), Seq(2), Seq("pos", "explode") ) runTest( - Inline('c.getField("h")), + Inline("c".attr.getField("h")), aliases => Inline($"${aliases(1)}".as("c.h")), - aliases => Seq('c.getField("d").as(aliases(0)), 'c.getField("h").as(aliases(1))), + aliases => Seq("c".attr.getField("d").as(aliases(0)), "c".attr.getField("h").as(aliases(1))), Seq(2), Seq("h1", "h2") ) } test("Column pruning for Project on Sort") { - val input = LocalRelation('a.int, 'b.string, 'c.double) + val input = LocalRelation("a".attr.int, "b".attr.string, "c".attr.double) - val query = input.orderBy('b.asc).select('a).analyze + val query = input.orderBy("b".attr.asc).select("a".attr).analyze val optimized = Optimize.execute(query) - val correctAnswer = input.select('a, 'b).orderBy('b.asc).select('a).analyze + val correctAnswer = + input.select("a".attr, "b".attr).orderBy("b".attr.asc).select("a".attr).analyze comparePlans(optimized, correctAnswer) } test("Column pruning for Expand") { - val input = LocalRelation('a.int, 'b.string, 'c.double) + val input = LocalRelation("a".attr.int, "b".attr.string, "c".attr.double) val query = Aggregate( - Seq('aa, 'gid), - Seq(sum('c).as("sum")), + Seq("aa".attr, "gid".attr), + Seq(sum("c".attr).as("sum")), Expand( Seq( - Seq('a, 'b, 'c, Literal.create(null, StringType), 1), - Seq('a, 'b, 'c, 'a, 2)), - Seq('a, 'b, 'c, 'aa.int, 'gid.int), + Seq("a".attr, "b".attr, "c".attr, Literal.create(null, StringType), 1), + Seq("a".attr, "b".attr, "c".attr, "a".attr, 2)), + Seq("a".attr, "b".attr, "c".attr, "aa".attr.int, "gid".attr.int), input)).analyze val optimized = Optimize.execute(query) val expected = Aggregate( - Seq('aa, 'gid), - Seq(sum('c).as("sum")), + Seq("aa".attr, "gid".attr), + Seq(sum("c".attr).as("sum")), Expand( Seq( - Seq('c, Literal.create(null, StringType), 1), - Seq('c, 'a, 2)), - Seq('c, 'aa.int, 'gid.int), - Project(Seq('a, 'c), + Seq("c".attr, Literal.create(null, StringType), 1), + Seq("c".attr, "a".attr, 2)), + Seq("c".attr, "aa".attr.int, "gid".attr.int), + Project(Seq("a".attr, "c".attr), input))).analyze comparePlans(optimized, expected) } test("Column pruning for ScriptTransformation") { - val input = LocalRelation('a.int, 'b.string, 'c.double) + val input = LocalRelation("a".attr.int, "b".attr.string, "c".attr.double) val query = ScriptTransformation( - Seq('a, 'b), + Seq("a".attr, "b".attr), "func", Seq.empty, input, @@ -231,11 +232,11 @@ class ColumnPruningSuite extends PlanTest { val expected = ScriptTransformation( - Seq('a, 'b), + Seq("a".attr, "b".attr), "func", Seq.empty, Project( - Seq('a, 'b), + Seq("a".attr, "b".attr), input), null).analyze @@ -243,34 +244,35 @@ class ColumnPruningSuite extends PlanTest { } test("Column pruning on Filter") { - val input = LocalRelation('a.int, 'b.string, 'c.double) - val plan1 = Filter('a > 1, input).analyze + val input = LocalRelation("a".attr.int, "b".attr.string, "c".attr.double) + val plan1 = Filter("a".attr > 1, input).analyze comparePlans(Optimize.execute(plan1), plan1) - val query = Project('a :: Nil, Filter('c > Literal(0.0), input)).analyze + val query = Project("a".attr :: Nil, Filter("c".attr > Literal(0.0), input)).analyze comparePlans(Optimize.execute(query), query) - val plan2 = Filter('b > 1, Project(Seq('a, 'b), input)).analyze - val expected2 = Project(Seq('a, 'b), Filter('b > 1, input)).analyze + val plan2 = Filter("b".attr > 1, Project(Seq("a".attr, "b".attr), input)).analyze + val expected2 = Project(Seq("a".attr, "b".attr), Filter("b".attr > 1, input)).analyze comparePlans(Optimize.execute(plan2), expected2) - val plan3 = Project(Seq('a), Filter('b > 1, Project(Seq('a, 'b), input))).analyze - val expected3 = Project(Seq('a), Filter('b > 1, input)).analyze + val plan3 = + Project(Seq("a".attr), Filter("b".attr > 1, Project(Seq("a".attr, "b".attr), input))).analyze + val expected3 = Project(Seq("a".attr), Filter("b".attr > 1, input)).analyze comparePlans(Optimize.execute(plan3), expected3) } test("Column pruning on except/intersect/distinct") { - val input = LocalRelation('a.int, 'b.string, 'c.double) - val query = Project('a :: Nil, Except(input, input, isAll = false)).analyze + val input = LocalRelation("a".attr.int, "b".attr.string, "c".attr.double) + val query = Project("a".attr :: Nil, Except(input, input, isAll = false)).analyze comparePlans(Optimize.execute(query), query) - val query2 = Project('a :: Nil, Intersect(input, input, isAll = false)).analyze + val query2 = Project("a".attr :: Nil, Intersect(input, input, isAll = false)).analyze comparePlans(Optimize.execute(query2), query2) - val query3 = Project('a :: Nil, Distinct(input)).analyze + val query3 = Project("a".attr :: Nil, Distinct(input)).analyze comparePlans(Optimize.execute(query3), query3) } test("Column pruning on Project") { - val input = LocalRelation('a.int, 'b.string, 'c.double) - val query = Project('a :: Nil, Project(Seq('a, 'b), input)).analyze - val expected = Project(Seq('a), input).analyze + val input = LocalRelation("a".attr.int, "b".attr.string, "c".attr.double) + val query = Project("a".attr :: Nil, Project(Seq("a".attr, "b".attr), input)).analyze + val expected = Project(Seq("a".attr), input).analyze comparePlans(Optimize.execute(query), expected) } @@ -291,140 +293,146 @@ class ColumnPruningSuite extends PlanTest { } test("column pruning for group") { - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) val originalQuery = testRelation - .groupBy('a)('a, count('b)) - .select('a) + .groupBy("a".attr)("a".attr, count("b".attr)) + .select("a".attr) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select('a) - .groupBy('a)('a).analyze + .select("a".attr) + .groupBy("a".attr)("a".attr).analyze comparePlans(optimized, correctAnswer) } test("column pruning for group with alias") { - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) val originalQuery = testRelation - .groupBy('a)('a as 'c, count('b)) - .select('c) + .groupBy("a".attr)("a".attr as "c", count("b".attr)) + .select("c".attr) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select('a) - .groupBy('a)('a as 'c).analyze + .select("a".attr) + .groupBy("a".attr)("a".attr as "c").analyze comparePlans(optimized, correctAnswer) } test("column pruning for Project(ne, Limit)") { - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) val originalQuery = testRelation - .select('a, 'b) + .select("a".attr, "b".attr) .limit(2) - .select('a) + .select("a".attr) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select('a) + .select("a".attr) .limit(2).analyze comparePlans(optimized, correctAnswer) } test("push down project past sort") { - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) - val x = testRelation.subquery('x) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) + val x = testRelation.subquery("x") // push down valid val originalQuery = { - x.select('a, 'b) - .sortBy(SortOrder('a, Ascending)) - .select('a) + x.select("a".attr, "b".attr) + .sortBy(SortOrder("a".attr, Ascending)) + .select("a".attr) } val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = - x.select('a) - .sortBy(SortOrder('a, Ascending)).analyze + x.select("a".attr) + .sortBy(SortOrder("a".attr, Ascending)).analyze comparePlans(optimized, correctAnswer) // push down invalid val originalQuery1 = { - x.select('a, 'b) - .sortBy(SortOrder('a, Ascending)) - .select('b) + x.select("a".attr, "b".attr) + .sortBy(SortOrder("a".attr, Ascending)) + .select("b".attr) } val optimized1 = Optimize.execute(originalQuery1.analyze) val correctAnswer1 = - x.select('a, 'b) - .sortBy(SortOrder('a, Ascending)) - .select('b).analyze + x.select("a".attr, "b".attr) + .sortBy(SortOrder("a".attr, Ascending)) + .select("b".attr).analyze comparePlans(optimized1, correctAnswer1) } test("Column pruning on Window with useless aggregate functions") { - val input = LocalRelation('a.int, 'b.string, 'c.double, 'd.int) - val winSpec = windowSpec('a :: Nil, 'd.asc :: Nil, UnspecifiedFrame) - val winExpr = windowExpr(count('d), winSpec) - - val originalQuery = input.groupBy('a, 'c, 'd)('a, 'c, 'd, winExpr.as('window)).select('a, 'c) - val correctAnswer = input.select('a, 'c, 'd).groupBy('a, 'c, 'd)('a, 'c).analyze + val input = LocalRelation("a".attr.int, "b".attr.string, "c".attr.double, "d".attr.int) + val winSpec = windowSpec("a".attr :: Nil, "d".attr.asc :: Nil, UnspecifiedFrame) + val winExpr = windowExpr(count("d".attr), winSpec) + + val originalQuery = input + .groupBy("a".attr, "c".attr, "d".attr)("a".attr, "c".attr, "d".attr, winExpr.as("window")) + .select("a".attr, "c".attr) + val correctAnswer = input.select("a".attr, "c".attr, "d".attr) + .groupBy("a".attr, "c".attr, "d".attr)("a".attr, "c".attr).analyze val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, correctAnswer) } test("Column pruning on Window with selected agg expressions") { - val input = LocalRelation('a.int, 'b.string, 'c.double, 'd.int) - val winSpec = windowSpec('a :: Nil, 'b.asc :: Nil, UnspecifiedFrame) - val winExpr = windowExpr(count('b), winSpec) + val input = LocalRelation("a".attr.int, "b".attr.string, "c".attr.double, "d".attr.int) + val winSpec = windowSpec("a".attr :: Nil, "b".attr.asc :: Nil, UnspecifiedFrame) + val winExpr = windowExpr(count("b".attr), winSpec) - val originalQuery = - input.select('a, 'b, 'c, 'd, winExpr.as('window)).where('window > 1).select('a, 'c) + val originalQuery = input + .select("a".attr, "b".attr, "c".attr, "d".attr, winExpr.as("window")) + .where("window".attr > 1).select("a".attr, "c".attr) val correctAnswer = - input.select('a, 'b, 'c) - .window(winExpr.as('window) :: Nil, 'a :: Nil, 'b.asc :: Nil) - .where('window > 1).select('a, 'c).analyze + input.select("a".attr, "b".attr, "c".attr) + .window(winExpr.as("window") :: Nil, "a".attr :: Nil, "b".attr.asc :: Nil) + .where("window".attr > 1).select("a".attr, "c".attr).analyze val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, correctAnswer) } test("Column pruning on Window in select") { - val input = LocalRelation('a.int, 'b.string, 'c.double, 'd.int) - val winSpec = windowSpec('a :: Nil, 'b.asc :: Nil, UnspecifiedFrame) - val winExpr = windowExpr(count('b), winSpec) + val input = LocalRelation("a".attr.int, "b".attr.string, "c".attr.double, "d".attr.int) + val winSpec = windowSpec("a".attr :: Nil, "b".attr.asc :: Nil, UnspecifiedFrame) + val winExpr = windowExpr(count("b".attr), winSpec) - val originalQuery = input.select('a, 'b, 'c, 'd, winExpr.as('window)).select('a, 'c) - val correctAnswer = input.select('a, 'c).analyze + val originalQuery = input.select("a".attr, "b".attr, "c".attr, "d".attr, + winExpr.as("window")).select("a".attr, "c".attr) + val correctAnswer = input.select("a".attr, "c".attr).analyze val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, correctAnswer) } test("Column pruning on Union") { - val input1 = LocalRelation('a.int, 'b.string, 'c.double) - val input2 = LocalRelation('c.int, 'd.string, 'e.double) - val query = Project('b :: Nil, Union(input1 :: input2 :: Nil)).analyze - val expected = Union(Project('b :: Nil, input1) :: Project('d :: Nil, input2) :: Nil).analyze + val input1 = LocalRelation("a".attr.int, "b".attr.string, "c".attr.double) + val input2 = LocalRelation("c".attr.int, "d".attr.string, "e".attr.double) + val query = Project("b".attr :: Nil, Union(input1 :: input2 :: Nil)).analyze + val expected = + Union(Project("b".attr :: Nil, input1) :: Project("d".attr :: Nil, input2) :: Nil).analyze comparePlans(Optimize.execute(query), expected) } test("Remove redundant projects in column pruning rule") { - val input = LocalRelation('key.int, 'value.string) + val input = LocalRelation("key".attr.int, "value".attr.string) val query = Project(Seq($"x.key", $"y.key"), @@ -447,33 +455,33 @@ class ColumnPruningSuite extends PlanTest { private val func = identity[Iterator[OtherTuple]] _ test("Column pruning on MapPartitions") { - val input = LocalRelation('_1.int, '_2.int, 'c.int) + val input = LocalRelation("_1".attr.int, "_2".attr.int, "c".attr.int) val plan1 = MapPartitions(func, input) val correctAnswer1 = - MapPartitions(func, Project(Seq('_1, '_2), input)).analyze + MapPartitions(func, Project(Seq("_1".attr, "_2".attr), input)).analyze comparePlans(Optimize.execute(plan1.analyze), correctAnswer1) } test("push project down into sample") { - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) - val x = testRelation.subquery('x) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) + val x = testRelation.subquery("x") - val query1 = Sample(0.0, 0.6, false, 11L, x).select('a) + val query1 = Sample(0.0, 0.6, false, 11L, x).select("a".attr) val optimized1 = Optimize.execute(query1.analyze) - val expected1 = Sample(0.0, 0.6, false, 11L, x.select('a)) + val expected1 = Sample(0.0, 0.6, false, 11L, x.select("a".attr)) comparePlans(optimized1, expected1.analyze) - val query2 = Sample(0.0, 0.6, false, 11L, x).select('a as 'aa) + val query2 = Sample(0.0, 0.6, false, 11L, x).select("a".attr as "aa") val optimized2 = Optimize.execute(query2.analyze) - val expected2 = Sample(0.0, 0.6, false, 11L, x.select('a as 'aa)) + val expected2 = Sample(0.0, 0.6, false, 11L, x.select("a".attr as "aa")) comparePlans(optimized2, expected2.analyze) } test("SPARK-24696 ColumnPruning rule fails to remove extra Project") { - val input = LocalRelation('key.int, 'value.string) - val query = input.select('key).where(rand(0L) > 0.5).where('key < 10).analyze + val input = LocalRelation("key".attr.int, "value".attr.string) + val query = input.select("key".attr).where(rand(0L) > 0.5).where("key".attr < 10).analyze val optimized = Optimize.execute(query) - val expected = input.where(rand(0L) > 0.5).where('key < 10).select('key).analyze + val expected = input.where(rand(0L) > 0.5).where("key".attr < 10).select("key".attr).analyze comparePlans(optimized, expected) } // todo: add more tests for column pruning diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala index 11f908ac180b..237ad682473d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala @@ -55,14 +55,14 @@ class CombiningLimitsSuite extends PlanTest { test("limits: combines two limits") { val originalQuery = testRelation - .select('a) + .select("a".attr) .limit(10) .limit(5) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select('a) + .select("a".attr) .limit(5).analyze comparePlans(optimized, correctAnswer) @@ -71,7 +71,7 @@ class CombiningLimitsSuite extends PlanTest { test("limits: combines three limits") { val originalQuery = testRelation - .select('a) + .select("a".attr) .limit(2) .limit(7) .limit(5) @@ -79,7 +79,7 @@ class CombiningLimitsSuite extends PlanTest { val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select('a) + .select("a".attr) .limit(2).analyze comparePlans(optimized, correctAnswer) @@ -88,15 +88,15 @@ class CombiningLimitsSuite extends PlanTest { test("limits: combines two limits after ColumnPruning") { val originalQuery = testRelation - .select('a) + .select("a".attr) .limit(2) - .select('a) + .select("a".attr) .limit(5) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select('a) + .select("a".attr) .limit(2).analyze comparePlans(optimized, correctAnswer) @@ -115,7 +115,7 @@ class CombiningLimitsSuite extends PlanTest { comparePlans(optimized2, query2) // test child max row is none - val query3 = testRelation.select(Symbol("a")).limit(1).analyze + val query3 = testRelation.select("a".attr).limit(1).analyze val optimized3 = Optimize.execute(query3) comparePlans(optimized3, query3) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index ae644c111074..6057a99bbac3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -38,18 +38,18 @@ class ConstantFoldingSuite extends PlanTest { BooleanSimplification) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) test("eliminate subqueries") { val originalQuery = testRelation - .subquery('y) - .select('a) + .subquery("y") + .select("a".attr) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select('a.attr) + .select("a".attr) .analyze comparePlans(optimized, correctAnswer) @@ -62,27 +62,27 @@ class ConstantFoldingSuite extends PlanTest { val originalQuery = testRelation .select( - Literal(2) + Literal(3) + Literal(4) as Symbol("2+3+4"), - Literal(2) * Literal(3) + Literal(4) as Symbol("2*3+4"), - Literal(2) * (Literal(3) + Literal(4)) as Symbol("2*(3+4)")) + Literal(2) + Literal(3) + Literal(4) as "2+3+4", + Literal(2) * Literal(3) + Literal(4) as "2*3+4", + Literal(2) * (Literal(3) + Literal(4)) as "2*(3+4)") .where( Literal(1) === Literal(1) && Literal(2) > Literal(3) || Literal(3) > Literal(2) ) .groupBy( Literal(2) * Literal(3) - Literal(6) / (Literal(4) - Literal(2)) - )(Literal(9) / Literal(3) as Symbol("9/3")) + )(Literal(9) / Literal(3) as "9/3") val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select( - Literal(9) as Symbol("2+3+4"), - Literal(10) as Symbol("2*3+4"), - Literal(14) as Symbol("2*(3+4)")) + Literal(9) as "2+3+4", + Literal(10) as "2*3+4", + Literal(14) as "2*(3+4)") .where(Literal(true)) - .groupBy(Literal(3.0))(Literal(3.0) as Symbol("9/3")) + .groupBy(Literal(3.0))(Literal(3.0) as "9/3") .analyze comparePlans(optimized, correctAnswer) @@ -93,20 +93,20 @@ class ConstantFoldingSuite extends PlanTest { val originalQuery = testRelation .select( - Literal(2) + Literal(3) + 'a as Symbol("c1"), - 'a + Literal(2) + Literal(3) as Symbol("c2"), - Literal(2) * 'a + Literal(4) as Symbol("c3"), - 'a * (Literal(3) + Literal(4)) as Symbol("c4")) + Literal(2) + Literal(3) + "a".attr as "c1", + "a".attr + Literal(2) + Literal(3) as "c2", + Literal(2) * "a".attr + Literal(4) as "c3", + "a".attr * (Literal(3) + Literal(4)) as "c4") val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select( - Literal(5) + 'a as Symbol("c1"), - 'a + Literal(2) + Literal(3) as Symbol("c2"), - Literal(2) * 'a + Literal(4) as Symbol("c3"), - 'a * Literal(7) as Symbol("c4")) + Literal(5) + "a".attr as "c1", + "a".attr + Literal(2) + Literal(3) as "c2", + Literal(2) * "a".attr + Literal(4) as "c3", + "a".attr * Literal(7) as "c4") .analyze comparePlans(optimized, correctAnswer) @@ -117,20 +117,20 @@ class ConstantFoldingSuite extends PlanTest { val originalQuery = testRelation .where( - (('a > 1 && Literal(1) === Literal(1)) || - ('a < 10 && Literal(1) === Literal(2)) || - (Literal(1) === Literal(1) && 'b > 1) || - (Literal(1) === Literal(2) && 'b < 10)) && - (('a > 1 || Literal(1) === Literal(1)) && - ('a < 10 || Literal(1) === Literal(2)) && - (Literal(1) === Literal(1) || 'b > 1) && - (Literal(1) === Literal(2) || 'b < 10))) + (("a".attr > 1 && Literal(1) === Literal(1)) || + ("a".attr < 10 && Literal(1) === Literal(2)) || + (Literal(1) === Literal(1) && "b".attr > 1) || + (Literal(1) === Literal(2) && "b".attr < 10)) && + (("a".attr > 1 || Literal(1) === Literal(1)) && + ("a".attr < 10 || Literal(1) === Literal(2)) && + (Literal(1) === Literal(1) || "b".attr > 1) && + (Literal(1) === Literal(2) || "b".attr < 10))) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where(('a > 1 || 'b > 1) && ('a < 10 && 'b < 10)) + .where(("a".attr > 1 || "b".attr > 1) && ("a".attr < 10 && "b".attr < 10)) .analyze comparePlans(optimized, correctAnswer) @@ -140,16 +140,16 @@ class ConstantFoldingSuite extends PlanTest { val originalQuery = testRelation .select( - Cast(Literal("2"), IntegerType) + Literal(3) + 'a as Symbol("c1"), - Coalesce(Seq(Cast(Literal("abc"), IntegerType), Literal(3))) as Symbol("c2")) + Cast(Literal("2"), IntegerType) + Literal(3) + "a".attr as "c1", + Coalesce(Seq(Cast(Literal("abc"), IntegerType), Literal(3))) as "c2") val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select( - Literal(5) + 'a as Symbol("c1"), - Literal(3) as Symbol("c2")) + Literal(5) + "a".attr as "c1", + Literal(3) as "c2") .analyze comparePlans(optimized, correctAnswer) @@ -159,16 +159,16 @@ class ConstantFoldingSuite extends PlanTest { val originalQuery = testRelation .select( - Rand(5L) + Literal(1) as Symbol("c1"), - sum('a) as Symbol("c2")) + Rand(5L) + Literal(1) as "c1", + sum("a".attr) as "c2") val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select( - Rand(5L) + Literal(1.0) as Symbol("c1"), - sum('a) as Symbol("c2")) + Rand(5L) + Literal(1.0) as "c1", + sum("a".attr) as "c2") .analyze comparePlans(optimized, correctAnswer) @@ -176,37 +176,37 @@ class ConstantFoldingSuite extends PlanTest { test("Constant folding test: expressions have null literals") { val originalQuery = testRelation.select( - IsNull(Literal(null)) as 'c1, - IsNotNull(Literal(null)) as 'c2, + IsNull(Literal(null)) as "c1", + IsNotNull(Literal(null)) as "c2", - UnresolvedExtractValue(Literal.create(null, ArrayType(IntegerType)), 1) as 'c3, + UnresolvedExtractValue(Literal.create(null, ArrayType(IntegerType)), 1) as "c3", UnresolvedExtractValue( - Literal.create(Seq(1), ArrayType(IntegerType)), Literal.create(null, IntegerType)) as 'c4, + Literal.create(Seq(1), ArrayType(IntegerType)), Literal.create(null, IntegerType)) as "c4", UnresolvedExtractValue( Literal.create(null, StructType(Seq(StructField("a", IntegerType, true)))), - "a") as 'c5, + "a") as "c5", - UnaryMinus(Literal.create(null, IntegerType)) as 'c6, - Cast(Literal(null), IntegerType) as 'c7, - Not(Literal.create(null, BooleanType)) as 'c8, + UnaryMinus(Literal.create(null, IntegerType)) as "c6", + Cast(Literal(null), IntegerType) as "c7", + Not(Literal.create(null, BooleanType)) as "c8", - Add(Literal.create(null, IntegerType), 1) as 'c9, - Add(1, Literal.create(null, IntegerType)) as 'c10, + Add(Literal.create(null, IntegerType), 1) as "c9", + Add(1, Literal.create(null, IntegerType)) as "c10", - EqualTo(Literal.create(null, IntegerType), 1) as 'c11, - EqualTo(1, Literal.create(null, IntegerType)) as 'c12, + EqualTo(Literal.create(null, IntegerType), 1) as "c11", + EqualTo(1, Literal.create(null, IntegerType)) as "c12", - new Like(Literal.create(null, StringType), "abc") as 'c13, - new Like("abc", Literal.create(null, StringType)) as 'c14, + new Like(Literal.create(null, StringType), "abc") as "c13", + new Like("abc", Literal.create(null, StringType)) as "c14", - Upper(Literal.create(null, StringType)) as 'c15, + Upper(Literal.create(null, StringType)) as "c15", - Substring(Literal.create(null, StringType), 0, 1) as 'c16, - Substring("abc", Literal.create(null, IntegerType), 1) as 'c17, - Substring("abc", 0, Literal.create(null, IntegerType)) as 'c18, + Substring(Literal.create(null, StringType), 0, 1) as "c16", + Substring("abc", Literal.create(null, IntegerType), 1) as "c17", + Substring("abc", 0, Literal.create(null, IntegerType)) as "c18", - Contains(Literal.create(null, StringType), "abc") as 'c19, - Contains("abc", Literal.create(null, StringType)) as 'c20 + Contains(Literal.create(null, StringType), "abc") as "c19", + Contains("abc", Literal.create(null, StringType)) as "c20" ) val optimized = Optimize.execute(originalQuery.analyze) @@ -214,34 +214,34 @@ class ConstantFoldingSuite extends PlanTest { val correctAnswer = testRelation .select( - Literal(true) as 'c1, - Literal(false) as 'c2, + Literal(true) as "c1", + Literal(false) as "c2", - Literal.create(null, IntegerType) as 'c3, - Literal.create(null, IntegerType) as 'c4, - Literal.create(null, IntegerType) as 'c5, + Literal.create(null, IntegerType) as "c3", + Literal.create(null, IntegerType) as "c4", + Literal.create(null, IntegerType) as "c5", - Literal.create(null, IntegerType) as 'c6, - Literal.create(null, IntegerType) as 'c7, - Literal.create(null, BooleanType) as 'c8, + Literal.create(null, IntegerType) as "c6", + Literal.create(null, IntegerType) as "c7", + Literal.create(null, BooleanType) as "c8", - Literal.create(null, IntegerType) as 'c9, - Literal.create(null, IntegerType) as 'c10, + Literal.create(null, IntegerType) as "c9", + Literal.create(null, IntegerType) as "c10", - Literal.create(null, BooleanType) as 'c11, - Literal.create(null, BooleanType) as 'c12, + Literal.create(null, BooleanType) as "c11", + Literal.create(null, BooleanType) as "c12", - Literal.create(null, BooleanType) as 'c13, - Literal.create(null, BooleanType) as 'c14, + Literal.create(null, BooleanType) as "c13", + Literal.create(null, BooleanType) as "c14", - Literal.create(null, StringType) as 'c15, + Literal.create(null, StringType) as "c15", - Literal.create(null, StringType) as 'c16, - Literal.create(null, StringType) as 'c17, - Literal.create(null, StringType) as 'c18, + Literal.create(null, StringType) as "c16", + Literal.create(null, StringType) as "c17", + Literal.create(null, StringType) as "c18", - Literal.create(null, BooleanType) as 'c19, - Literal.create(null, BooleanType) as 'c20 + Literal.create(null, BooleanType) as "c19", + Literal.create(null, BooleanType) as "c20" ).analyze comparePlans(optimized, correctAnswer) @@ -250,14 +250,14 @@ class ConstantFoldingSuite extends PlanTest { test("Constant folding test: Fold In(v, list) into true or false") { val originalQuery = testRelation - .select('a) + .select("a".attr) .where(In(Literal(1), Seq(Literal(1), Literal(2)))) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select('a) + .select("a".attr) .where(Literal(true)) .analyze @@ -267,7 +267,7 @@ class ConstantFoldingSuite extends PlanTest { test("SPARK-33544: Constant folding test with side effects") { val originalQuery = testRelation - .select('a) + .select("a".attr) .where(Size(CreateArray(Seq(AssertTrue(false)))) > 0) val optimized = Optimize.execute(originalQuery.analyze) @@ -287,14 +287,14 @@ class ConstantFoldingSuite extends PlanTest { test("SPARK-33544: Constant folding test CreateArray") { val originalQuery = testRelation - .select('a) - .where(Size(CreateArray(Seq('a))) > 0) + .select("a".attr) + .where(Size(CreateArray(Seq("a".attr))) > 0) val optimized = OptimizeForCreate.execute(originalQuery.analyze) val correctAnswer = testRelation - .select('a) + .select("a".attr) .analyze comparePlans(optimized, correctAnswer) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantPropagationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantPropagationSuite.scala index 171ac4e3091c..a4f3e7ebf69a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantPropagationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantPropagationSuite.scala @@ -40,12 +40,12 @@ class ConstantPropagationSuite extends PlanTest { BooleanSimplification) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int, 'c.int, 'd.int.notNull) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int, "d".attr.int.notNull) - private val columnA = 'a - private val columnB = 'b - private val columnC = 'c - private val columnD = 'd + private val columnA = "a".attr + private val columnB = "b".attr + private val columnC = "c".attr + private val columnD = "d".attr test("basic test") { val query = testRelation diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala index 43579d4c903a..49e360ff7ffa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala @@ -39,11 +39,11 @@ class ConvertToLocalRelationSuite extends PlanTest { test("Project on LocalRelation should be turned into a single LocalRelation") { val testRelation = LocalRelation( - LocalRelation('a.int, 'b.int).output, + LocalRelation("a".attr.int, "b".attr.int).output, InternalRow(1, 2) :: InternalRow(4, 5) :: Nil) val correctAnswer = LocalRelation( - LocalRelation('a1.int, 'b1.int).output, + LocalRelation("a1".attr.int, "b1".attr.int).output, InternalRow(1, 3) :: InternalRow(4, 6) :: Nil) val projectOnLocal = testRelation.select( @@ -57,11 +57,11 @@ class ConvertToLocalRelationSuite extends PlanTest { test("Filter on LocalRelation should be turned into a single LocalRelation") { val testRelation = LocalRelation( - LocalRelation('a.int, 'b.int).output, + LocalRelation("a".attr.int, "b".attr.int).output, InternalRow(1, 2) :: InternalRow(4, 5) :: Nil) val correctAnswer = LocalRelation( - LocalRelation('a1.int, 'b1.int).output, + LocalRelation("a1".attr.int, "b1".attr.int).output, InternalRow(1, 3) :: Nil) val filterAndProjectOnLocal = testRelation @@ -75,11 +75,11 @@ class ConvertToLocalRelationSuite extends PlanTest { test("SPARK-27798: Expression reusing output shouldn't override values in local relation") { val testRelation = LocalRelation( - LocalRelation('a.int).output, + LocalRelation("a".attr.int).output, InternalRow(1) :: InternalRow(2) :: Nil) val correctAnswer = LocalRelation( - LocalRelation('a.struct('a1.int)).output, + LocalRelation("a".attr.struct("a1".attr.int)).output, InternalRow(InternalRow(1)) :: InternalRow(InternalRow(2)) :: Nil) val projected = testRelation.select(ExprReuseOutput(UnresolvedAttribute("a")).as("a")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/DecimalAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/DecimalAggregatesSuite.scala index 711294ed6192..4a400f60451e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/DecimalAggregatesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/DecimalAggregatesSuite.scala @@ -32,19 +32,19 @@ class DecimalAggregatesSuite extends PlanTest { DecimalAggregates) :: Nil } - val testRelation = LocalRelation('a.decimal(2, 1), 'b.decimal(12, 1)) + val testRelation = LocalRelation("a".attr.decimal(2, 1), "b".attr.decimal(12, 1)) test("Decimal Sum Aggregation: Optimized") { - val originalQuery = testRelation.select(sum('a)) + val originalQuery = testRelation.select(sum("a".attr)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select(MakeDecimal(sum(UnscaledValue('a)), 12, 1).as("sum(a)")).analyze + .select(MakeDecimal(sum(UnscaledValue("a".attr)), 12, 1).as("sum(a)")).analyze comparePlans(optimized, correctAnswer) } test("Decimal Sum Aggregation: Not Optimized") { - val originalQuery = testRelation.select(sum('b)) + val originalQuery = testRelation.select(sum("b".attr)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = originalQuery.analyze @@ -52,16 +52,16 @@ class DecimalAggregatesSuite extends PlanTest { } test("Decimal Average Aggregation: Optimized") { - val originalQuery = testRelation.select(avg('a)) + val originalQuery = testRelation.select(avg("a".attr)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select((avg(UnscaledValue('a)) / 10.0).cast(DecimalType(6, 5)).as("avg(a)")).analyze + .select((avg(UnscaledValue("a".attr)) / 10.0).cast(DecimalType(6, 5)).as("avg(a)")).analyze comparePlans(optimized, correctAnswer) } test("Decimal Average Aggregation: Not Optimized") { - val originalQuery = testRelation.select(avg('b)) + val originalQuery = testRelation.select(avg("b".attr)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = originalQuery.analyze @@ -69,25 +69,25 @@ class DecimalAggregatesSuite extends PlanTest { } test("Decimal Sum Aggregation over Window: Optimized") { - val spec = windowSpec(Seq('a), Nil, UnspecifiedFrame) - val originalQuery = testRelation.select(windowExpr(sum('a), spec).as('sum_a)) + val spec = windowSpec(Seq("a".attr), Nil, UnspecifiedFrame) + val originalQuery = testRelation.select(windowExpr(sum("a".attr), spec).as("sum_a")) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select('a) + .select("a".attr) .window( - Seq(MakeDecimal(windowExpr(sum(UnscaledValue('a)), spec), 12, 1).as('sum_a)), - Seq('a), + Seq(MakeDecimal(windowExpr(sum(UnscaledValue("a".attr)), spec), 12, 1).as("sum_a")), + Seq("a".attr), Nil) - .select('a, 'sum_a, 'sum_a) - .select('sum_a) + .select("a".attr, "sum_a".attr, "sum_a".attr) + .select("sum_a".attr) .analyze comparePlans(optimized, correctAnswer) } test("Decimal Sum Aggregation over Window: Not Optimized") { - val spec = windowSpec('b :: Nil, Nil, UnspecifiedFrame) - val originalQuery = testRelation.select(windowExpr(sum('b), spec)) + val spec = windowSpec("b".attr :: Nil, Nil, UnspecifiedFrame) + val originalQuery = testRelation.select(windowExpr(sum("b".attr), spec)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = originalQuery.analyze @@ -95,25 +95,26 @@ class DecimalAggregatesSuite extends PlanTest { } test("Decimal Average Aggregation over Window: Optimized") { - val spec = windowSpec(Seq('a), Nil, UnspecifiedFrame) - val originalQuery = testRelation.select(windowExpr(avg('a), spec).as('avg_a)) + val spec = windowSpec(Seq("a".attr), Nil, UnspecifiedFrame) + val originalQuery = testRelation.select(windowExpr(avg("a".attr), spec).as("avg_a")) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select('a) + .select("a".attr) .window( - Seq((windowExpr(avg(UnscaledValue('a)), spec) / 10.0).cast(DecimalType(6, 5)).as('avg_a)), - Seq('a), + Seq((windowExpr( + avg(UnscaledValue("a".attr)), spec) / 10.0).cast(DecimalType(6, 5)).as("avg_a")), + Seq("a".attr), Nil) - .select('a, 'avg_a, 'avg_a) - .select('avg_a) + .select("a".attr, "avg_a".attr, "avg_a".attr) + .select("avg_a".attr) .analyze comparePlans(optimized, correctAnswer) } test("Decimal Average Aggregation over Window: Not Optimized") { - val spec = windowSpec('b :: Nil, Nil, UnspecifiedFrame) - val originalQuery = testRelation.select(windowExpr(avg('b), spec)) + val spec = windowSpec("b".attr :: Nil, Nil, UnspecifiedFrame) + val originalQuery = testRelation.select(windowExpr(avg("b".attr), spec)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = originalQuery.analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateAggregateFilterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateAggregateFilterSuite.scala index ec9b876f78e1..c07cc40dabd2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateAggregateFilterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateAggregateFilterSuite.scala @@ -30,44 +30,44 @@ class EliminateAggregateFilterSuite extends PlanTest { Batch("Operator Optimizations", Once, ConstantFolding, EliminateAggregateFilter) :: Nil } - val testRelation = LocalRelation('a.int) + val testRelation = LocalRelation("a".attr.int) test("Eliminate Filter always is true") { val query = testRelation - .select(sumDistinct('a, Some(Literal.TrueLiteral)).as('result)) + .select(sumDistinct("a".attr, Some(Literal.TrueLiteral)).as("result")) .analyze val answer = testRelation - .select(sumDistinct('a).as('result)) + .select(sumDistinct("a".attr).as("result")) .analyze comparePlans(Optimize.execute(query), answer) } test("Eliminate Filter is foldable and always is true") { val query = testRelation - .select(countDistinctWithFilter(GreaterThan(Literal(2), Literal(1)), 'a).as('result)) + .select(countDistinctWithFilter(GreaterThan(Literal(2), Literal(1)), "a".attr).as("result")) .analyze val answer = testRelation - .select(countDistinct('a).as('result)) + .select(countDistinct("a".attr).as("result")) .analyze comparePlans(Optimize.execute(query), answer) } test("Eliminate Filter always is false") { val query = testRelation - .select(sumDistinct('a, Some(Literal.FalseLiteral)).as('result)) + .select(sumDistinct("a".attr, Some(Literal.FalseLiteral)).as("result")) .analyze val answer = testRelation - .groupBy()(Literal.create(null, LongType).as('result)) + .groupBy()(Literal.create(null, LongType).as("result")) .analyze comparePlans(Optimize.execute(query), answer) } test("Eliminate Filter is foldable and always is false") { val query = testRelation - .select(countDistinctWithFilter(GreaterThan(Literal(1), Literal(2)), 'a).as('result)) + .select(countDistinctWithFilter(GreaterThan(Literal(1), Literal(2)), "a".attr).as("result")) .analyze val answer = testRelation - .groupBy()(Literal.create(0L, LongType).as('result)) + .groupBy()(Literal.create(0L, LongType).as("result")) .analyze comparePlans(Optimize.execute(query), answer) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateDistinctSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateDistinctSuite.scala index 0848d5609ff0..2c0661c9728c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateDistinctSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateDistinctSuite.scala @@ -32,7 +32,7 @@ class EliminateDistinctSuite extends PlanTest { EliminateDistinct) :: Nil } - val testRelation = LocalRelation('a.int) + val testRelation = LocalRelation("a".attr.int) Seq( Max(_), @@ -42,13 +42,13 @@ class EliminateDistinctSuite extends PlanTest { CollectSet(_: Expression) ).foreach { aggBuilder => - val agg = aggBuilder('a) + val agg = aggBuilder("a".attr) test(s"Eliminate Distinct in ${agg.prettyName}") { val query = testRelation - .select(agg.toAggregateExpression(isDistinct = true).as('result)) + .select(agg.toAggregateExpression(isDistinct = true).as("result")) .analyze val answer = testRelation - .select(agg.toAggregateExpression(isDistinct = false).as('result)) + .select(agg.toAggregateExpression(isDistinct = false).as("result")) .analyze assert(query != answer) comparePlans(Optimize.execute(query), answer) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateMapObjectsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateMapObjectsSuite.scala index 157472c2293f..cf0988b35a00 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateMapObjectsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateMapObjectsSuite.scala @@ -42,7 +42,7 @@ class EliminateMapObjectsSuite extends PlanTest { test("SPARK-20254: Remove unnecessary data conversion for primitive array") { val intObjType = ObjectType(classOf[Array[Int]]) - val intInput = LocalRelation('a.array(ArrayType(IntegerType, false))) + val intInput = LocalRelation("a".attr.array(ArrayType(IntegerType, false))) val intQuery = intInput.deserialize[Array[Int]].analyze val intOptimized = Optimize.execute(intQuery) val intExpected = DeserializeToObject( @@ -51,7 +51,7 @@ class EliminateMapObjectsSuite extends PlanTest { comparePlans(intOptimized, intExpected) val doubleObjType = ObjectType(classOf[Array[Double]]) - val doubleInput = LocalRelation('a.array(ArrayType(DoubleType, false))) + val doubleInput = LocalRelation("a".attr.array(ArrayType(DoubleType, false))) val doubleQuery = doubleInput.deserialize[Array[Double]].analyze val doubleOptimized = Optimize.execute(doubleQuery) val doubleExpected = DeserializeToObject( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSerializationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSerializationSuite.scala index ef38cc076d95..f9ea7532df19 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSerializationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSerializationSuite.scala @@ -39,22 +39,22 @@ class EliminateSerializationSuite extends PlanTest { implicit private def intEncoder = ExpressionEncoder[Int]() test("back to back serialization") { - val input = LocalRelation('obj.obj(classOf[(Int, Int)])) + val input = LocalRelation("obj".attr.obj(classOf[(Int, Int)])) val plan = input.serialize[(Int, Int)].deserialize[(Int, Int)].analyze val optimized = Optimize.execute(plan) - val expected = input.select('obj.as("obj")).analyze + val expected = input.select("obj".attr.as("obj")).analyze comparePlans(optimized, expected) } test("back to back serialization with object change") { - val input = LocalRelation('obj.obj(classOf[OtherTuple])) + val input = LocalRelation("obj".attr.obj(classOf[OtherTuple])) val plan = input.serialize[OtherTuple].deserialize[(Int, Int)].analyze val optimized = Optimize.execute(plan) comparePlans(optimized, plan) } test("back to back serialization in AppendColumns") { - val input = LocalRelation('obj.obj(classOf[(Int, Int)])) + val input = LocalRelation("obj".attr.obj(classOf[(Int, Int)])) val func = (item: (Int, Int)) => item._1 val plan = AppendColumns(func, input.serialize[(Int, Int)]).analyze @@ -70,7 +70,7 @@ class EliminateSerializationSuite extends PlanTest { } test("back to back serialization in AppendColumns with object change") { - val input = LocalRelation('obj.obj(classOf[OtherTuple])) + val input = LocalRelation("obj".attr.obj(classOf[OtherTuple])) val func = (item: (Int, Int)) => item._1 val plan = AppendColumns(func, input.serialize[OtherTuple]).analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala index 82db174ad41b..b015eb5bba90 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsBeforeRepartitionSuite.scala @@ -30,8 +30,8 @@ class EliminateSortsBeforeRepartitionSuite extends PlanTest { val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry) val analyzer = new Analyzer(catalog) - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) - val anotherTestRelation = LocalRelation('d.int, 'e.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) + val anotherTestRelation = LocalRelation("d".attr.int, "e".attr.int) object Optimize extends RuleExecutor[LogicalPlan] { val batches = @@ -47,87 +47,105 @@ class EliminateSortsBeforeRepartitionSuite extends PlanTest { def repartition(plan: LogicalPlan): LogicalPlan = plan.repartition(10) test("sortBy") { - val plan = testRelation.select('a, 'b).sortBy('a.asc, 'b.desc) - val optimizedPlan = testRelation.select('a, 'b) + val plan = testRelation.select("a".attr, "b".attr).sortBy("a".attr.asc, "b".attr.desc) + val optimizedPlan = testRelation.select("a".attr, "b".attr) checkRepartitionCases(plan, optimizedPlan) } test("sortBy with projection") { - val plan = testRelation.sortBy('a.asc, 'b.asc).select('a + 1 as "a", 'b + 2 as "b") - val optimizedPlan = testRelation.select('a + 1 as "a", 'b + 2 as "b") + val plan = testRelation + .sortBy("a".attr.asc, "b".attr.asc) + .select("a".attr + 1 as "a", "b".attr + 2 as "b") + val optimizedPlan = testRelation.select("a".attr + 1 as "a", "b".attr + 2 as "b") checkRepartitionCases(plan, optimizedPlan) } test("sortBy with projection and filter") { - val plan = testRelation.sortBy('a.asc, 'b.asc).select('a, 'b).where('a === 10) - val optimizedPlan = testRelation.select('a, 'b).where('a === 10) + val plan = testRelation + .sortBy("a".attr.asc, "b".attr.asc) + .select("a".attr, "b".attr).where("a".attr === 10) + val optimizedPlan = testRelation.select("a".attr, "b".attr).where("a".attr === 10) checkRepartitionCases(plan, optimizedPlan) } test("sortBy with limit") { - val plan = testRelation.sortBy('a.asc, 'b.asc).limit(10) - val optimizedPlan = testRelation.sortBy('a.asc, 'b.asc).limit(10) + val plan = testRelation.sortBy("a".attr.asc, "b".attr.asc).limit(10) + val optimizedPlan = testRelation.sortBy("a".attr.asc, "b".attr.asc).limit(10) checkRepartitionCases(plan, optimizedPlan) } test("sortBy with non-deterministic projection") { - val plan = testRelation.sortBy('a.asc, 'b.asc).select(rand(1), 'a, 'b) - val optimizedPlan = testRelation.sortBy('a.asc, 'b.asc).select(rand(1), 'a, 'b) + val plan = testRelation.sortBy("a".attr.asc, "b".attr.asc).select(rand(1), "a".attr, "b".attr) + val optimizedPlan = + testRelation.sortBy("a".attr.asc, "b".attr.asc).select(rand(1), "a".attr, "b".attr) checkRepartitionCases(plan, optimizedPlan) } test("orderBy") { - val plan = testRelation.select('a, 'b).orderBy('a.asc, 'b.asc) - val optimizedPlan = testRelation.select('a, 'b) + val plan = testRelation.select("a".attr, "b".attr).orderBy("a".attr.asc, "b".attr.asc) + val optimizedPlan = testRelation.select("a".attr, "b".attr) checkRepartitionCases(plan, optimizedPlan) } test("orderBy with projection") { - val plan = testRelation.orderBy('a.asc, 'b.asc).select('a + 1 as "a", 'b + 2 as "b") - val optimizedPlan = testRelation.select('a + 1 as "a", 'b + 2 as "b") + val plan = testRelation + .orderBy("a".attr.asc, "b".attr.asc) + .select("a".attr + 1 as "a", "b".attr + 2 as "b") + val optimizedPlan = testRelation.select("a".attr + 1 as "a", "b".attr + 2 as "b") checkRepartitionCases(plan, optimizedPlan) } test("orderBy with projection and filter") { - val plan = testRelation.orderBy('a.asc, 'b.asc).select('a, 'b).where('a === 10) - val optimizedPlan = testRelation.select('a, 'b).where('a === 10) + val plan = testRelation + .orderBy("a".attr.asc, "b".attr.asc) + .select("a".attr, "b".attr) + .where("a".attr === 10) + val optimizedPlan = testRelation.select("a".attr, "b".attr).where("a".attr === 10) checkRepartitionCases(plan, optimizedPlan) } test("orderBy with limit") { - val plan = testRelation.orderBy('a.asc, 'b.asc).limit(10) - val optimizedPlan = testRelation.orderBy('a.asc, 'b.asc).limit(10) + val plan = testRelation.orderBy("a".attr.asc, "b".attr.asc).limit(10) + val optimizedPlan = testRelation.orderBy("a".attr.asc, "b".attr.asc).limit(10) checkRepartitionCases(plan, optimizedPlan) } test("orderBy with non-deterministic projection") { - val plan = testRelation.orderBy('a.asc, 'b.asc).select(rand(1), 'a, 'b) - val optimizedPlan = testRelation.orderBy('a.asc, 'b.asc).select(rand(1), 'a, 'b) + val plan = testRelation.orderBy("a".attr.asc, "b".attr.asc).select(rand(1), "a".attr, "b".attr) + val optimizedPlan = + testRelation.orderBy("a".attr.asc, "b".attr.asc).select(rand(1), "a".attr, "b".attr) checkRepartitionCases(plan, optimizedPlan) } test("additional coalesce and sortBy") { - val plan = testRelation.sortBy('a.asc, 'b.asc).coalesce(1) + val plan = testRelation.sortBy("a".attr.asc, "b".attr.asc).coalesce(1) val optimizedPlan = testRelation.coalesce(1) checkRepartitionCases(plan, optimizedPlan) } test("additional projection, repartition and sortBy") { - val plan = testRelation.sortBy('a.asc, 'b.asc).repartition(100).select('a + 1 as "a") - val optimizedPlan = testRelation.repartition(100).select('a + 1 as "a") + val plan = + testRelation.sortBy("a".attr.asc, "b".attr.asc).repartition(100).select("a".attr + 1 as "a") + val optimizedPlan = testRelation.repartition(100).select("a".attr + 1 as "a") checkRepartitionCases(plan, optimizedPlan) } test("additional filter, distribute and sortBy") { - val plan = testRelation.sortBy('a.asc, 'b.asc).distribute('a)(2).where('a === 10) - val optimizedPlan = testRelation.distribute('a)(2).where('a === 10) + val plan = testRelation + .sortBy("a".attr.asc, "b".attr.asc) + .distribute("a".attr)(2) + .where("a".attr === 10) + val optimizedPlan = testRelation.distribute("a".attr)(2).where("a".attr === 10) checkRepartitionCases(plan, optimizedPlan) } test("join") { - val plan = testRelation.sortBy('a.asc, 'b.asc).distribute('a)(2).where('a === 10) - val optimizedPlan = testRelation.distribute('a)(2).where('a === 10) - val anotherPlan = anotherTestRelation.select('d) + val plan = testRelation + .sortBy("a".attr.asc, "b".attr.asc) + .distribute("a".attr)(2) + .where("a".attr === 10) + val optimizedPlan = testRelation.distribute("a".attr)(2).where("a".attr === 10) + val anotherPlan = anotherTestRelation.select("d".attr) val joinPlan = plan.join(anotherPlan) val optimizedJoinPlan = optimize(joinPlan) val correctJoinPlan = analyze(optimizedPlan.join(anotherPlan)) @@ -135,11 +153,12 @@ class EliminateSortsBeforeRepartitionSuite extends PlanTest { } test("aggregate") { - val plan = testRelation.sortBy('a.asc, 'b.asc).distribute('a)(2).where('a === 10) - val optimizedPlan = testRelation.distribute('a)(2).where('a === 10) - val aggPlan = plan.groupBy('a)(sum('b)) + val plan = + testRelation.sortBy("a".attr.asc, "b".attr.asc).distribute("a".attr)(2).where("a".attr === 10) + val optimizedPlan = testRelation.distribute("a".attr)(2).where("a".attr === 10) + val aggPlan = plan.groupBy("a".attr)(sum("b".attr)) val optimizedAggPlan = optimize(aggPlan) - val correctAggPlan = analyze(optimizedPlan.groupBy('a)(sum('b))) + val correctAggPlan = analyze(optimizedPlan.groupBy("a".attr)(sum("b".attr))) comparePlans(optimizedAggPlan, correctAggPlan) } @@ -151,15 +170,17 @@ class EliminateSortsBeforeRepartitionSuite extends PlanTest { comparePlans(optimizedPlanWithRepartition, correctPlanWithRepartition) // can remove sortBy before repartition with sortBy - val planWithRepartitionAndSortBy = planWithRepartition.sortBy('a.asc) + val planWithRepartitionAndSortBy = planWithRepartition.sortBy("a".attr.asc) val optimizedPlanWithRepartitionAndSortBy = optimize(planWithRepartitionAndSortBy) - val correctPlanWithRepartitionAndSortBy = analyze(repartition(optimizedPlan).sortBy('a.asc)) + val correctPlanWithRepartitionAndSortBy = + analyze(repartition(optimizedPlan).sortBy("a".attr.asc)) comparePlans(optimizedPlanWithRepartitionAndSortBy, correctPlanWithRepartitionAndSortBy) // can remove sortBy before repartition with orderBy - val planWithRepartitionAndOrderBy = planWithRepartition.orderBy('a.asc) + val planWithRepartitionAndOrderBy = planWithRepartition.orderBy("a".attr.asc) val optimizedPlanWithRepartitionAndOrderBy = optimize(planWithRepartitionAndOrderBy) - val correctPlanWithRepartitionAndOrderBy = analyze(repartition(optimizedPlan).orderBy('a.asc)) + val correctPlanWithRepartitionAndOrderBy = + analyze(repartition(optimizedPlan).orderBy("a".attr.asc)) comparePlans(optimizedPlanWithRepartitionAndOrderBy, correctPlanWithRepartitionAndOrderBy) } @@ -173,17 +194,17 @@ class EliminateSortsBeforeRepartitionSuite extends PlanTest { } class EliminateSortsBeforeRepartitionByExprsSuite extends EliminateSortsBeforeRepartitionSuite { - override def repartition(plan: LogicalPlan): LogicalPlan = plan.distribute('a)(10) + override def repartition(plan: LogicalPlan): LogicalPlan = plan.distribute("a".attr)(10) test("sortBy before repartition with non-deterministic expressions") { - val plan = testRelation.sortBy('a.asc, 'b.asc).limit(10) - val planWithRepartition = plan.distribute(rand(1).asc, 'a.asc)(20) + val plan = testRelation.sortBy("a".attr.asc, "b".attr.asc).limit(10) + val planWithRepartition = plan.distribute(rand(1).asc, "a".attr.asc)(20) checkRepartitionCases(plan = planWithRepartition, optimizedPlan = planWithRepartition) } test("orderBy before repartition with non-deterministic expressions") { - val plan = testRelation.orderBy('a.asc, 'b.asc).limit(10) - val planWithRepartition = plan.distribute(rand(1).asc, 'a.asc)(20) + val plan = testRelation.orderBy("a".attr.asc, "b".attr.asc).limit(10) + val planWithRepartition = plan.distribute(rand(1).asc, "a".attr.asc)(20) checkRepartitionCases(plan = planWithRepartition, optimizedPlan = planWithRepartition) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala index 01ecbd808c25..36b9352f75f7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala @@ -86,9 +86,9 @@ class EliminateSortsSuite extends AnalysisTest { val x = testRelation val analyzer = getAnalyzer - val query = x.orderBy(SortOrder(3, Ascending), 'a.asc) + val query = x.orderBy(SortOrder(3, Ascending), "a".attr.asc) val optimized = Optimize.execute(analyzer.execute(query)) - val correctAnswer = analyzer.execute(x.orderBy('a.asc)) + val correctAnswer = analyzer.execute(x.orderBy("a".attr.asc)) comparePlans(optimized, correctAnswer) } @@ -97,11 +97,12 @@ class EliminateSortsSuite extends AnalysisTest { test("Remove no-op alias") { val x = testRelation - val query = x.select('a.as('x), Year(CurrentDate()).as('y), 'b) - .orderBy('x.asc, 'y.asc, 'b.desc) + val query = x.select("a".attr.as("x"), Year(CurrentDate()).as("y"), "b".attr) + .orderBy("x".attr.asc, "y".attr.asc, "b".attr.desc) val optimized = Optimize.execute(analyzer.execute(query)) val correctAnswer = analyzer.execute( - x.select('a.as('x), Year(CurrentDate()).as('y), 'b).orderBy('x.asc, 'b.desc)) + x.select("a".attr.as("x"), Year(CurrentDate()).as("y"), "b".attr) + .orderBy("x".attr.asc, "b".attr.desc)) comparePlans(optimized, correctAnswer) } @@ -114,72 +115,82 @@ class EliminateSortsSuite extends AnalysisTest { } test("SPARK-33183: remove redundant sort by") { - val orderedPlan = testRelation.select('a, 'b).orderBy('a.asc, 'b.desc_nullsFirst) - val unnecessaryReordered = orderedPlan.limit(2).select('a).sortBy('a.asc, 'b.desc_nullsFirst) + val orderedPlan = + testRelation.select("a".attr, "b".attr).orderBy("a".attr.asc, "b".attr.desc_nullsFirst) + val unnecessaryReordered = orderedPlan + .limit(2).select("a".attr).sortBy("a".attr.asc, "b".attr.desc_nullsFirst) val optimized = Optimize.execute(unnecessaryReordered.analyze) - val correctAnswer = orderedPlan.limit(2).select('a).analyze + val correctAnswer = orderedPlan.limit(2).select("a".attr).analyze comparePlans(optimized, correctAnswer) } test("SPARK-33183: remove all redundant local sorts") { - val orderedPlan = testRelation.sortBy('a.asc).orderBy('a.asc).sortBy('a.asc) + val orderedPlan = testRelation.sortBy("a".attr.asc).orderBy("a".attr.asc).sortBy("a".attr.asc) val optimized = Optimize.execute(orderedPlan.analyze) - val correctAnswer = testRelation.orderBy('a.asc).analyze + val correctAnswer = testRelation.orderBy("a".attr.asc).analyze comparePlans(optimized, correctAnswer) } test("SPARK-33183: should not remove global sort") { - val orderedPlan = testRelation.select('a, 'b).orderBy('a.asc, 'b.desc_nullsFirst) - val reordered = orderedPlan.limit(2).select('a).orderBy('a.asc, 'b.desc_nullsFirst) + val orderedPlan = + testRelation.select("a".attr, "b".attr).orderBy("a".attr.asc, "b".attr.desc_nullsFirst) + val reordered = + orderedPlan.limit(2).select("a".attr).orderBy("a".attr.asc, "b".attr.desc_nullsFirst) val optimized = Optimize.execute(reordered.analyze) val correctAnswer = reordered.analyze comparePlans(optimized, correctAnswer) } test("do not remove sort if the order is different") { - val orderedPlan = testRelation.select('a, 'b).orderBy('a.asc, 'b.desc_nullsFirst) - val reorderedDifferently = orderedPlan.limit(2).select('a).orderBy('a.asc, 'b.desc) + val orderedPlan = + testRelation.select("a".attr, "b".attr).orderBy("a".attr.asc, "b".attr.desc_nullsFirst) + val reorderedDifferently = + orderedPlan.limit(2).select("a".attr).orderBy("a".attr.asc, "b".attr.desc) val optimized = Optimize.execute(reorderedDifferently.analyze) val correctAnswer = reorderedDifferently.analyze comparePlans(optimized, correctAnswer) } test("SPARK-33183: remove top level local sort with filter operators") { - val orderedPlan = testRelation.select('a, 'b).orderBy('a.asc, 'b.desc) - val filteredAndReordered = orderedPlan.where('a > Literal(10)).sortBy('a.asc, 'b.desc) + val orderedPlan = testRelation.select("a".attr, "b".attr).orderBy("a".attr.asc, "b".attr.desc) + val filteredAndReordered = + orderedPlan.where("a".attr > Literal(10)).sortBy("a".attr.asc, "b".attr.desc) val optimized = Optimize.execute(filteredAndReordered.analyze) - val correctAnswer = orderedPlan.where('a > Literal(10)).analyze + val correctAnswer = orderedPlan.where("a".attr > Literal(10)).analyze comparePlans(optimized, correctAnswer) } test("SPARK-33183: keep top level global sort with filter operators") { - val projectPlan = testRelation.select('a, 'b) - val orderedPlan = projectPlan.orderBy('a.asc, 'b.desc) - val filteredAndReordered = orderedPlan.where('a > Literal(10)).orderBy('a.asc, 'b.desc) + val projectPlan = testRelation.select("a".attr, "b".attr) + val orderedPlan = projectPlan.orderBy("a".attr.asc, "b".attr.desc) + val filteredAndReordered = + orderedPlan.where("a".attr > Literal(10)).orderBy("a".attr.asc, "b".attr.desc) val optimized = Optimize.execute(filteredAndReordered.analyze) - val correctAnswer = projectPlan.where('a > Literal(10)).orderBy('a.asc, 'b.desc).analyze + val correctAnswer = + projectPlan.where("a".attr > Literal(10)).orderBy("a".attr.asc, "b".attr.desc).analyze comparePlans(optimized, correctAnswer) } test("SPARK-33183: limits should not affect order for local sort") { - val orderedPlan = testRelation.select('a, 'b).orderBy('a.asc, 'b.desc) - val filteredAndReordered = orderedPlan.limit(Literal(10)).sortBy('a.asc, 'b.desc) + val orderedPlan = testRelation.select("a".attr, "b".attr).orderBy("a".attr.asc, "b".attr.desc) + val filteredAndReordered = orderedPlan.limit(Literal(10)).sortBy("a".attr.asc, "b".attr.desc) val optimized = Optimize.execute(filteredAndReordered.analyze) val correctAnswer = orderedPlan.limit(Literal(10)).analyze comparePlans(optimized, correctAnswer) } test("SPARK-33183: should not remove global sort with limit operators") { - val orderedPlan = testRelation.select('a, 'b).orderBy('a.asc, 'b.desc) - val filteredAndReordered = orderedPlan.limit(Literal(10)).orderBy('a.asc, 'b.desc) + val orderedPlan = testRelation.select("a".attr, "b".attr).orderBy("a".attr.asc, "b".attr.desc) + val filteredAndReordered = orderedPlan.limit(Literal(10)).orderBy("a".attr.asc, "b".attr.desc) val optimized = Optimize.execute(filteredAndReordered.analyze) val correctAnswer = filteredAndReordered.analyze comparePlans(optimized, correctAnswer) } test("different sorts are not simplified if limit is in between") { - val orderedPlan = testRelation.select('a, 'b).orderBy('b.desc).limit(Literal(10)) - .orderBy('a.asc) + val orderedPlan = + testRelation.select("a".attr, "b".attr).orderBy("b".attr.desc).limit(Literal(10)) + .orderBy("a".attr.asc) val optimized = Optimize.execute(orderedPlan.analyze) val correctAnswer = orderedPlan.analyze comparePlans(optimized, correctAnswer) @@ -187,18 +198,18 @@ class EliminateSortsSuite extends AnalysisTest { test("SPARK-33183: should not remove global sort with range operator") { val inputPlan = Range(1L, 1000L, 1, 10) - val orderedPlan = inputPlan.orderBy('id.asc) + val orderedPlan = inputPlan.orderBy("id".attr.asc) val optimized = Optimize.execute(orderedPlan.analyze) val correctAnswer = orderedPlan.analyze comparePlans(optimized, correctAnswer) - val reversedPlan = inputPlan.orderBy('id.desc) + val reversedPlan = inputPlan.orderBy("id".attr.desc) val reversedOptimized = Optimize.execute(reversedPlan.analyze) val reversedCorrectAnswer = reversedPlan.analyze comparePlans(reversedOptimized, reversedCorrectAnswer) val negativeStepInputPlan = Range(10L, 1L, -1, 10) - val negativeStepOrderedPlan = negativeStepInputPlan.orderBy('id.desc) + val negativeStepOrderedPlan = negativeStepInputPlan.orderBy("id".attr.desc) val negativeStepOptimized = Optimize.execute(negativeStepOrderedPlan.analyze) val negativeStepCorrectAnswer = negativeStepOrderedPlan.analyze comparePlans(negativeStepOptimized, negativeStepCorrectAnswer) @@ -206,50 +217,56 @@ class EliminateSortsSuite extends AnalysisTest { test("SPARK-33183: remove local sort with range operator") { val inputPlan = Range(1L, 1000L, 1, 10) - val orderedPlan = inputPlan.sortBy('id.asc) + val orderedPlan = inputPlan.sortBy("id".attr.asc) val optimized = Optimize.execute(orderedPlan.analyze) val correctAnswer = inputPlan.analyze comparePlans(optimized, correctAnswer) } test("sort should not be removed when there is a node which doesn't guarantee any order") { - val orderedPlan = testRelation.select('a, 'b) - val groupedAndResorted = orderedPlan.groupBy('a)(sum('a)).orderBy('a.asc) + val orderedPlan = testRelation.select("a".attr, "b".attr) + val groupedAndResorted = orderedPlan.groupBy("a".attr)(sum("a".attr)).orderBy("a".attr.asc) val optimized = Optimize.execute(groupedAndResorted.analyze) val correctAnswer = groupedAndResorted.analyze comparePlans(optimized, correctAnswer) } test("remove two consecutive sorts") { - val orderedTwice = testRelation.orderBy('a.asc).orderBy('b.desc) + val orderedTwice = testRelation.orderBy("a".attr.asc).orderBy("b".attr.desc) val optimized = Optimize.execute(orderedTwice.analyze) - val correctAnswer = testRelation.orderBy('b.desc).analyze + val correctAnswer = testRelation.orderBy("b".attr.desc).analyze comparePlans(optimized, correctAnswer) } test("remove sorts separated by Filter/Project operators") { - val orderedTwiceWithProject = testRelation.orderBy('a.asc).select('b).orderBy('b.desc) + val orderedTwiceWithProject = + testRelation.orderBy("a".attr.asc).select("b".attr).orderBy("b".attr.desc) val optimizedWithProject = Optimize.execute(orderedTwiceWithProject.analyze) - val correctAnswerWithProject = testRelation.select('b).orderBy('b.desc).analyze + val correctAnswerWithProject = + testRelation.select("b".attr).orderBy("b".attr.desc).analyze comparePlans(optimizedWithProject, correctAnswerWithProject) val orderedTwiceWithFilter = - testRelation.orderBy('a.asc).where('b > Literal(0)).orderBy('b.desc) + testRelation.orderBy("a".attr.asc).where("b".attr > Literal(0)).orderBy("b".attr.desc) val optimizedWithFilter = Optimize.execute(orderedTwiceWithFilter.analyze) - val correctAnswerWithFilter = testRelation.where('b > Literal(0)).orderBy('b.desc).analyze + val correctAnswerWithFilter = + testRelation.where("b".attr > Literal(0)).orderBy("b".attr.desc).analyze comparePlans(optimizedWithFilter, correctAnswerWithFilter) - val orderedTwiceWithBoth = - testRelation.orderBy('a.asc).select('b).where('b > Literal(0)).orderBy('b.desc) + val orderedTwiceWithBoth = testRelation + .orderBy("a".attr.asc) + .select("b".attr) + .where("b".attr > Literal(0)) + .orderBy("b".attr.desc) val optimizedWithBoth = Optimize.execute(orderedTwiceWithBoth.analyze) val correctAnswerWithBoth = - testRelation.select('b).where('b > Literal(0)).orderBy('b.desc).analyze + testRelation.select("b".attr).where("b".attr > Literal(0)).orderBy("b".attr.desc).analyze comparePlans(optimizedWithBoth, correctAnswerWithBoth) - val orderedThrice = orderedTwiceWithBoth.select(('b + 1).as('c)).orderBy('c.asc) + val orderedThrice = orderedTwiceWithBoth.select(("b".attr + 1).as("c")).orderBy("c".attr.asc) val optimizedThrice = Optimize.execute(orderedThrice.analyze) - val correctAnswerThrice = testRelation.select('b).where('b > Literal(0)) - .select(('b + 1).as('c)).orderBy('c.asc).analyze + val correctAnswerThrice = testRelation.select("b".attr).where("b".attr > Literal(0)) + .select(("b".attr + 1).as("c")).orderBy("c".attr.asc).analyze comparePlans(optimizedThrice, correctAnswerThrice) } @@ -265,37 +282,37 @@ class EliminateSortsSuite extends AnalysisTest { (e : Expression) => bitOr(e), (e : Expression) => bitXor(e) ).foreach(agg => { - val projectPlan = testRelation.select('a, 'b) - val unnecessaryOrderByPlan = projectPlan.orderBy('a.asc, 'b.desc) - val groupByPlan = unnecessaryOrderByPlan.groupBy('a)(agg('b)) + val projectPlan = testRelation.select("a".attr, "b".attr) + val unnecessaryOrderByPlan = projectPlan.orderBy("a".attr.asc, "b".attr.desc) + val groupByPlan = unnecessaryOrderByPlan.groupBy("a".attr)(agg("b".attr)) val optimized = Optimize.execute(groupByPlan.analyze) - val correctAnswer = projectPlan.groupBy('a)(agg('b)).analyze + val correctAnswer = projectPlan.groupBy("a".attr)(agg("b".attr)).analyze comparePlans(optimized, correctAnswer) }) } test("remove orderBy in groupBy clause with sum aggs") { - val projectPlan = testRelation.select('a, 'b) - val unnecessaryOrderByPlan = projectPlan.orderBy('a.asc, 'b.desc) - val groupByPlan = unnecessaryOrderByPlan.groupBy('a)(sum('a) + 10 as "sum") + val projectPlan = testRelation.select("a".attr, "b".attr) + val unnecessaryOrderByPlan = projectPlan.orderBy("a".attr.asc, "b".attr.desc) + val groupByPlan = unnecessaryOrderByPlan.groupBy("a".attr)(sum("a".attr) + 10 as "sum") val optimized = Optimize.execute(groupByPlan.analyze) - val correctAnswer = projectPlan.groupBy('a)(sum('a) + 10 as "sum").analyze + val correctAnswer = projectPlan.groupBy("a".attr)(sum("a".attr) + 10 as "sum").analyze comparePlans(optimized, correctAnswer) } test("should not remove orderBy in groupBy clause with first aggs") { - val projectPlan = testRelation.select('a, 'b) - val orderByPlan = projectPlan.orderBy('a.asc, 'b.desc) - val groupByPlan = orderByPlan.groupBy('a)(first('a)) + val projectPlan = testRelation.select("a".attr, "b".attr) + val orderByPlan = projectPlan.orderBy("a".attr.asc, "b".attr.desc) + val groupByPlan = orderByPlan.groupBy("a".attr)(first("a".attr)) val optimized = Optimize.execute(groupByPlan.analyze) val correctAnswer = groupByPlan.analyze comparePlans(optimized, correctAnswer) } test("should not remove orderBy in groupBy clause with first and count aggs") { - val projectPlan = testRelation.select('a, 'b) - val orderByPlan = projectPlan.orderBy('a.asc, 'b.desc) - val groupByPlan = orderByPlan.groupBy('a)(first('a), count(1)) + val projectPlan = testRelation.select("a".attr, "b".attr) + val orderByPlan = projectPlan.orderBy("a".attr.asc, "b".attr.desc) + val groupByPlan = orderByPlan.groupBy("a".attr)(first("a".attr), count(1)) val optimized = Optimize.execute(groupByPlan.analyze) val correctAnswer = groupByPlan.analyze comparePlans(optimized, correctAnswer) @@ -304,67 +321,67 @@ class EliminateSortsSuite extends AnalysisTest { test("should not remove orderBy in groupBy clause with PythonUDF as aggs") { val pythonUdf = PythonUDF("pyUDF", null, IntegerType, Seq.empty, PythonEvalType.SQL_BATCHED_UDF, true) - val projectPlan = testRelation.select('a, 'b) - val orderByPlan = projectPlan.orderBy('a.asc, 'b.desc) - val groupByPlan = orderByPlan.groupBy('a)(pythonUdf) + val projectPlan = testRelation.select("a".attr, "b".attr) + val orderByPlan = projectPlan.orderBy("a".attr.asc, "b".attr.desc) + val groupByPlan = orderByPlan.groupBy("a".attr)(pythonUdf) val optimized = Optimize.execute(groupByPlan.analyze) val correctAnswer = groupByPlan.analyze comparePlans(optimized, correctAnswer) } test("should not remove orderBy in groupBy clause with ScalaUDF as aggs") { - val scalaUdf = ScalaUDF((s: Int) => s, IntegerType, 'a :: Nil, + val scalaUdf = ScalaUDF((s: Int) => s, IntegerType, "a".attr :: Nil, Option(ExpressionEncoder[Int]()) :: Nil) - val projectPlan = testRelation.select('a, 'b) - val orderByPlan = projectPlan.orderBy('a.asc, 'b.desc) - val groupByPlan = orderByPlan.groupBy('a)(scalaUdf) + val projectPlan = testRelation.select("a".attr, "b".attr) + val orderByPlan = projectPlan.orderBy("a".attr.asc, "b".attr.desc) + val groupByPlan = orderByPlan.groupBy("a".attr)(scalaUdf) val optimized = Optimize.execute(groupByPlan.analyze) val correctAnswer = groupByPlan.analyze comparePlans(optimized, correctAnswer) } test("should not remove orderBy with limit in groupBy clause") { - val projectPlan = testRelation.select('a, 'b) - val orderByPlan = projectPlan.orderBy('a.asc, 'b.desc).limit(10) - val groupByPlan = orderByPlan.groupBy('a)(count(1)) + val projectPlan = testRelation.select("a".attr, "b".attr) + val orderByPlan = projectPlan.orderBy("a".attr.asc, "b".attr.desc).limit(10) + val groupByPlan = orderByPlan.groupBy("a".attr)(count(1)) val optimized = Optimize.execute(groupByPlan.analyze) val correctAnswer = groupByPlan.analyze comparePlans(optimized, correctAnswer) } test("remove orderBy in join clause") { - val projectPlan = testRelation.select('a, 'b) - val unnecessaryOrderByPlan = projectPlan.orderBy('a.asc, 'b.desc) - val projectPlanB = testRelationB.select('d) - val joinPlan = unnecessaryOrderByPlan.join(projectPlanB).select('a, 'd) + val projectPlan = testRelation.select("a".attr, "b".attr) + val unnecessaryOrderByPlan = projectPlan.orderBy("a".attr.asc, "b".attr.desc) + val projectPlanB = testRelationB.select("d".attr) + val joinPlan = unnecessaryOrderByPlan.join(projectPlanB).select("a".attr, "d".attr) val optimized = Optimize.execute(joinPlan.analyze) - val correctAnswer = projectPlan.join(projectPlanB).select('a, 'd).analyze + val correctAnswer = projectPlan.join(projectPlanB).select("a".attr, "d".attr).analyze comparePlans(optimized, correctAnswer) } test("should not remove orderBy with limit in join clause") { - val projectPlan = testRelation.select('a, 'b) - val orderByPlan = projectPlan.orderBy('a.asc, 'b.desc).limit(10) - val projectPlanB = testRelationB.select('d) - val joinPlan = orderByPlan.join(projectPlanB).select('a, 'd) + val projectPlan = testRelation.select("a".attr, "b".attr) + val orderByPlan = projectPlan.orderBy("a".attr.asc, "b".attr.desc).limit(10) + val projectPlanB = testRelationB.select("d".attr) + val joinPlan = orderByPlan.join(projectPlanB).select("a".attr, "d".attr) val optimized = Optimize.execute(joinPlan.analyze) val correctAnswer = joinPlan.analyze comparePlans(optimized, correctAnswer) } test("SPARK-32318: should not remove orderBy in distribute statement") { - val projectPlan = testRelation.select('a, 'b) - val orderByPlan = projectPlan.orderBy('b.desc) - val distributedPlan = orderByPlan.distribute('a)(1) + val projectPlan = testRelation.select("a".attr, "b".attr) + val orderByPlan = projectPlan.orderBy("b".attr.desc) + val distributedPlan = orderByPlan.distribute("a".attr)(1) val optimized = Optimize.execute(distributedPlan.analyze) val correctAnswer = distributedPlan.analyze comparePlans(optimized, correctAnswer) } test("should not remove orderBy in left join clause if there is an outer limit") { - val projectPlan = testRelation.select('a, 'b) - val orderByPlan = projectPlan.orderBy('a.asc, 'b.desc) - val projectPlanB = testRelationB.select('d) + val projectPlan = testRelation.select("a".attr, "b".attr) + val orderByPlan = projectPlan.orderBy("a".attr.asc, "b".attr.desc) + val projectPlanB = testRelationB.select("d".attr) val joinPlan = orderByPlan .join(projectPlanB, LeftOuter) .limit(10) @@ -374,9 +391,9 @@ class EliminateSortsSuite extends AnalysisTest { } test("remove orderBy in right join clause event if there is an outer limit") { - val projectPlan = testRelation.select('a, 'b) - val orderByPlan = projectPlan.orderBy('a.asc, 'b.desc) - val projectPlanB = testRelationB.select('d) + val projectPlan = testRelation.select("a".attr, "b".attr) + val orderByPlan = projectPlan.orderBy("a".attr.asc, "b".attr.desc) + val projectPlanB = testRelationB.select("d".attr) val joinPlan = orderByPlan .join(projectPlanB, RightOuter) .limit(10) @@ -390,8 +407,10 @@ class EliminateSortsSuite extends AnalysisTest { test("SPARK-33183: remove consecutive global sorts with the same ordering") { Seq( - (testRelation.orderBy('a.asc).orderBy('a.asc), testRelation.orderBy('a.asc)), - (testRelation.orderBy('a.asc, 'b.desc).orderBy('a.asc), testRelation.orderBy('a.asc)) + (testRelation.orderBy("a".attr.asc).orderBy("a".attr.asc), + testRelation.orderBy("a".attr.asc)), + (testRelation.orderBy("a".attr.asc, "b".attr.desc).orderBy("a".attr.asc), + testRelation.orderBy("a".attr.asc)) ).foreach { case (ordered, answer) => val optimized = Optimize.execute(ordered.analyze) comparePlans(optimized, answer.analyze) @@ -399,24 +418,24 @@ class EliminateSortsSuite extends AnalysisTest { } test("SPARK-33183: remove consecutive local sorts with the same ordering") { - val orderedPlan = testRelation.sortBy('a.asc).sortBy('a.asc).sortBy('a.asc) + val orderedPlan = testRelation.sortBy("a".attr.asc).sortBy("a".attr.asc).sortBy("a".attr.asc) val optimized = Optimize.execute(orderedPlan.analyze) - val correctAnswer = testRelation.sortBy('a.asc).analyze + val correctAnswer = testRelation.sortBy("a".attr.asc).analyze comparePlans(optimized, correctAnswer) } test("SPARK-33183: remove consecutive local sorts with different ordering") { - val orderedPlan = testRelation.sortBy('b.asc).sortBy('a.desc).sortBy('a.asc) + val orderedPlan = testRelation.sortBy("b".attr.asc).sortBy("a".attr.desc).sortBy("a".attr.asc) val optimized = Optimize.execute(orderedPlan.analyze) - val correctAnswer = testRelation.sortBy('a.asc).analyze + val correctAnswer = testRelation.sortBy("a".attr.asc).analyze comparePlans(optimized, correctAnswer) } test("SPARK-33183: should keep global sort when child is a local sort with the same ordering") { - val correctAnswer = testRelation.orderBy('a.asc).analyze + val correctAnswer = testRelation.orderBy("a".attr.asc).analyze Seq( - testRelation.sortBy('a.asc).orderBy('a.asc), - testRelation.orderBy('a.asc).sortBy('a.asc).orderBy('a.asc) + testRelation.sortBy("a".attr.asc).orderBy("a".attr.asc), + testRelation.orderBy("a".attr.asc).sortBy("a".attr.asc).orderBy("a".attr.asc) ).foreach { ordered => val optimized = Optimize.execute(ordered.analyze) comparePlans(optimized, correctAnswer) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSubqueryAliasesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSubqueryAliasesSuite.scala index 4df1a145a271..cc72fa4d34ec 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSubqueryAliasesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSubqueryAliasesSuite.scala @@ -45,25 +45,25 @@ class EliminateSubqueryAliasesSuite extends PlanTest with PredicateHelper { } test("eliminate top level subquery") { - val input = LocalRelation('a.int, 'b.int) + val input = LocalRelation("a".attr.int, "b".attr.int) val query = SubqueryAlias("a", input) comparePlans(afterOptimization(query), input) } test("eliminate mid-tree subquery") { - val input = LocalRelation('a.int, 'b.int) + val input = LocalRelation("a".attr.int, "b".attr.int) val query = Filter(TrueLiteral, SubqueryAlias("a", input)) comparePlans( afterOptimization(query), - Filter(TrueLiteral, LocalRelation('a.int, 'b.int))) + Filter(TrueLiteral, LocalRelation("a".attr.int, "b".attr.int))) } test("eliminate multiple subqueries") { - val input = LocalRelation('a.int, 'b.int) + val input = LocalRelation("a".attr.int, "b".attr.int) val query = Filter(TrueLiteral, SubqueryAlias("c", SubqueryAlias("b", SubqueryAlias("a", input)))) comparePlans( afterOptimization(query), - Filter(TrueLiteral, LocalRelation('a.int, 'b.int))) + Filter(TrueLiteral, LocalRelation("a".attr.int, "b".attr.int))) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExtractPythonUDFFromJoinConditionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExtractPythonUDFFromJoinConditionSuite.scala index 77bfc0b3682a..6795c42ff91f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExtractPythonUDFFromJoinConditionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExtractPythonUDFFromJoinConditionSuite.scala @@ -38,10 +38,10 @@ class ExtractPythonUDFFromJoinConditionSuite extends PlanTest { CheckCartesianProducts) :: Nil } - val attrA = 'a.int - val attrB = 'b.int - val attrC = 'c.int - val attrD = 'd.int + val attrA = "a".attr.int + val attrB = "b".attr.int + val attrC = "c".attr.int + val attrD = "d".attr.int val testRelationLeft = LocalRelation(attrA, attrB) val testRelationRight = LocalRelation(attrC, attrD) @@ -105,11 +105,11 @@ class ExtractPythonUDFFromJoinConditionSuite extends PlanTest { val query = testRelationLeft.join( testRelationRight, joinType = Inner, - condition = Some(unevaluableJoinCond && 'a.attr === 'c.attr)) + condition = Some(unevaluableJoinCond && "a".attr === "c".attr)) val expected = testRelationLeft.join( testRelationRight, joinType = Inner, - condition = Some('a.attr === 'c.attr)).where(unevaluableJoinCond).analyze + condition = Some("a".attr === "c".attr)).where(unevaluableJoinCond).analyze val optimized = Optimize.execute(query.analyze) comparePlans(optimized, expected) } @@ -118,11 +118,11 @@ class ExtractPythonUDFFromJoinConditionSuite extends PlanTest { val query = testRelationLeft.join( testRelationRight, joinType = Inner, - condition = Some(unevaluableJoinCond || 'a.attr === 'c.attr)) + condition = Some(unevaluableJoinCond || "a".attr === "c".attr)) val expected = testRelationLeft.join( testRelationRight, joinType = Inner, - condition = None).where(unevaluableJoinCond || 'a.attr === 'c.attr).analyze + condition = None).where(unevaluableJoinCond || "a".attr === "c".attr).analyze comparePlanWithCrossJoinEnable(query, expected) } @@ -132,7 +132,7 @@ class ExtractPythonUDFFromJoinConditionSuite extends PlanTest { Seq(attrA, attrC), PythonEvalType.SQL_BATCHED_UDF, udfDeterministic = true) - val condition = (unevaluableJoinCond || 'a.attr === 'c.attr) && pythonUDF1 + val condition = (unevaluableJoinCond || "a".attr === "c".attr) && pythonUDF1 val query = testRelationLeft.join( testRelationRight, @@ -151,7 +151,7 @@ class ExtractPythonUDFFromJoinConditionSuite extends PlanTest { Seq(attrA, attrC), PythonEvalType.SQL_BATCHED_UDF, udfDeterministic = true) - val condition = (unevaluableJoinCond || pythonUDF1) && 'a.attr === 'c.attr + val condition = (unevaluableJoinCond || pythonUDF1) && "a".attr === "c".attr val query = testRelationLeft.join( testRelationRight, @@ -160,7 +160,8 @@ class ExtractPythonUDFFromJoinConditionSuite extends PlanTest { val expected = testRelationLeft.join( testRelationRight, joinType = Inner, - condition = Some('a.attr === 'c.attr)).where(unevaluableJoinCond || pythonUDF1).analyze + condition = + Some("a".attr === "c".attr)).where(unevaluableJoinCond || pythonUDF1).analyze val optimized = Optimize.execute(query.analyze) comparePlans(optimized, expected) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownOnePassSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownOnePassSuite.scala index 6f1280c90e9d..8debf0a888d0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownOnePassSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownOnePassSuite.scala @@ -42,12 +42,12 @@ class FilterPushdownOnePassSuite extends PlanTest { ) :: Nil } - val testRelation1 = LocalRelation('a.int, 'b.int, 'c.int) - val testRelation2 = LocalRelation('a.int, 'd.int, 'e.int) + val testRelation1 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) + val testRelation2 = LocalRelation("a".attr.int, "d".attr.int, "e".attr.int) test("really simple predicate push down") { - val x = testRelation1.subquery('x) - val y = testRelation2.subquery('y) + val x = testRelation1.subquery("x") + val y = testRelation2.subquery("y") val originalQuery = x.join(y).where("x.a".attr === 1) @@ -58,8 +58,8 @@ class FilterPushdownOnePassSuite extends PlanTest { } test("push down conjunctive predicates") { - val x = testRelation1.subquery('x) - val y = testRelation2.subquery('y) + val x = testRelation1.subquery("x") + val y = testRelation2.subquery("y") val originalQuery = x.join(y).where("x.a".attr === 1 && "y.d".attr < 1) @@ -70,8 +70,8 @@ class FilterPushdownOnePassSuite extends PlanTest { } test("push down predicates for simple joins") { - val x = testRelation1.subquery('x) - val y = testRelation2.subquery('y) + val x = testRelation1.subquery("x") + val y = testRelation2.subquery("y") val originalQuery = x.where("x.c".attr < 0) @@ -87,8 +87,8 @@ class FilterPushdownOnePassSuite extends PlanTest { } test("push down top-level filters for cascading joins") { - val x = testRelation1.subquery('x) - val y = testRelation2.subquery('y) + val x = testRelation1.subquery("x") + val y = testRelation2.subquery("y") val originalQuery = y.join(x).join(x).join(x).join(x).join(x).where("y.d".attr === 0) @@ -100,9 +100,9 @@ class FilterPushdownOnePassSuite extends PlanTest { } test("push down predicates for tree-like joins") { - val x = testRelation1.subquery('x) - val y1 = testRelation2.subquery('y1) - val y2 = testRelation2.subquery('y2) + val x = testRelation1.subquery("x") + val y1 = testRelation2.subquery("y1") + val y2 = testRelation2.subquery("y2") val originalQuery = y1.join(x).join(x) @@ -118,64 +118,64 @@ class FilterPushdownOnePassSuite extends PlanTest { } test("push down through join and project") { - val x = testRelation1.subquery('x) - val y = testRelation2.subquery('y) + val x = testRelation1.subquery("x") + val y = testRelation2.subquery("y") val originalQuery = - x.where('a > 0).select('a, 'b) - .join(y.where('d < 100).select('e)) + x.where("a".attr > 0).select("a".attr, "b".attr) + .join(y.where("d".attr < 100).select("e".attr)) .where("x.a".attr < 100) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = - x.where('a > 0 && 'a < 100).select('a, 'b) - .join(y.where('d < 100).select('e)).analyze + x.where("a".attr > 0 && "a".attr < 100).select("a".attr, "b".attr) + .join(y.where("d".attr < 100).select("e".attr)).analyze comparePlans(optimized, correctAnswer) } test("push down through deep projects") { - val x = testRelation1.subquery('x) + val x = testRelation1.subquery("x") val originalQuery = - x.select(('a + 1) as 'a1, 'b) - .select(('a1 + 1) as 'a2, 'b) - .select(('a2 + 1) as 'a3, 'b) - .select(('a3 + 1) as 'a4, 'b) - .select('b) - .where('b > 0) + x.select(("a".attr + 1)as "a1", "b".attr) + .select(("a1".attr + 1)as "a2", "b".attr) + .select(("a2".attr + 1)as "a3", "b".attr) + .select(("a3".attr + 1)as "a4", "b".attr) + .select("b".attr) + .where("b".attr > 0) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = - x.where('b > 0) - .select(('a + 1) as 'a1, 'b) - .select(('a1 + 1) as 'a2, 'b) - .select(('a2 + 1) as 'a3, 'b) - .select(('a3 + 1) as 'a4, 'b) - .select('b).analyze + x.where("b".attr > 0) + .select(("a".attr + 1)as "a1", "b".attr) + .select(("a1".attr + 1)as "a2", "b".attr) + .select(("a2".attr + 1)as "a3", "b".attr) + .select(("a3".attr + 1)as "a4", "b".attr) + .select("b".attr).analyze comparePlans(optimized, correctAnswer) } test("push down through aggregate and join") { - val x = testRelation1.subquery('x) - val y = testRelation2.subquery('y) + val x = testRelation1.subquery("x") + val y = testRelation2.subquery("y") val left = x - .where('c > 0) - .groupBy('a)('a, count('b)) - .subquery('left) + .where("c".attr > 0) + .groupBy("a".attr)("a".attr, count("b".attr)) + .subquery("left") val right = y - .where('d < 0) - .groupBy('a)('a, count('d)) - .subquery('right) + .where("d".attr < 0) + .groupBy("a".attr)("a".attr, count("d".attr)) + .subquery("right") val originalQuery = left .join(right).where("left.a".attr < 100 && "right.a".attr < 100) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = - x.where('c > 0 && 'a < 100).groupBy('a)('a, count('b)) - .join(y.where('d < 0 && 'a < 100).groupBy('a)('a, count('d))) + x.where("c".attr > 0 && "a".attr < 100).groupBy("a".attr)("a".attr, count("b".attr)) + .join(y.where("d".attr < 0 && "a".attr < 100).groupBy("a".attr)("a".attr, count("d".attr))) .analyze comparePlans(optimized, correctAnswer) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index c518fdded211..294a85b48f10 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -46,10 +46,10 @@ class FilterPushdownSuite extends PlanTest { PushDownPredicates) :: Nil } - val attrA = 'a.int - val attrB = 'b.int - val attrC = 'c.int - val attrD = 'd.int + val attrA = "a".attr.int + val attrB = "b".attr.int + val attrC = "c".attr.int + val attrD = "d".attr.int val testRelation = LocalRelation(attrA, attrB, attrC) @@ -58,8 +58,8 @@ class FilterPushdownSuite extends PlanTest { val simpleDisjunctivePredicate = ("x.a".attr > 3) && ("y.a".attr > 13) || ("x.a".attr > 1) && ("y.a".attr > 11) val expectedPredicatePushDownResult = { - val left = testRelation.where(('a > 3 || 'a > 1)).subquery('x) - val right = testRelation.where('a > 13 || 'a > 11).subquery('y) + val left = testRelation.where(("a".attr > 3 || "a".attr > 1)).subquery("x") + val right = testRelation.where("a".attr > 13 || "a".attr > 11).subquery("y") left.join(right, condition = Some("x.b".attr === "y.b".attr && (("x.a".attr > 3) && ("y.a".attr > 13) || ("x.a".attr > 1) && ("y.a".attr > 11)))).analyze } @@ -68,13 +68,13 @@ class FilterPushdownSuite extends PlanTest { test("eliminate subqueries") { val originalQuery = testRelation - .subquery('y) - .select('a) + .subquery("y") + .select("a".attr) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select('a.attr) + .select("a".attr) .analyze comparePlans(optimized, correctAnswer) @@ -84,14 +84,14 @@ class FilterPushdownSuite extends PlanTest { test("simple push down") { val originalQuery = testRelation - .select('a) - .where('a === 1) + .select("a".attr) + .where("a".attr === 1) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where('a === 1) - .select('a) + .where("a".attr === 1) + .select("a".attr) .analyze comparePlans(optimized, correctAnswer) @@ -100,13 +100,13 @@ class FilterPushdownSuite extends PlanTest { test("combine redundant filters") { val originalQuery = testRelation - .where('a === 1 && 'b === 1) - .where('a === 1 && 'c === 1) + .where("a".attr === 1 && "b".attr === 1) + .where("a".attr === 1 && "c".attr === 1) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where('a === 1 && 'b === 1 && 'c === 1) + .where("a".attr === 1 && "b".attr === 1 && "c".attr === 1) .analyze comparePlans(optimized, correctAnswer) @@ -115,8 +115,8 @@ class FilterPushdownSuite extends PlanTest { test("do not combine non-deterministic filters even if they are identical") { val originalQuery = testRelation - .where(Rand(0) > 0.1 && 'a === 1) - .where(Rand(0) > 0.1 && 'a === 1).analyze + .where(Rand(0) > 0.1 && "a".attr === 1) + .where(Rand(0) > 0.1 && "a".attr === 1).analyze val optimized = Optimize.execute(originalQuery) @@ -126,15 +126,15 @@ class FilterPushdownSuite extends PlanTest { test("SPARK-16164: Filter pushdown should keep the ordering in the logical plan") { val originalQuery = testRelation - .where('a === 1) - .select('a, 'b) - .where('b === 1) + .where("a".attr === 1) + .select("a".attr, "b".attr) + .where("b".attr === 1) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where('a === 1 && 'b === 1) - .select('a, 'b) + .where("a".attr === 1 && "b".attr === 1) + .select("a".attr, "b".attr) .analyze // We can not use comparePlans here because it normalized the plan. @@ -142,7 +142,7 @@ class FilterPushdownSuite extends PlanTest { } test("SPARK-16994: filter should not be pushed through limit") { - val originalQuery = testRelation.limit(10).where('a === 1).analyze + val originalQuery = testRelation.limit(10).where("a".attr === 1).analyze val optimized = Optimize.execute(originalQuery) comparePlans(optimized, originalQuery) } @@ -150,15 +150,15 @@ class FilterPushdownSuite extends PlanTest { test("can't push without rewrite") { val originalQuery = testRelation - .select('a + 'b as 'e) - .where('e === 1) + .select("a".attr + "b".attr as "e") + .where("e".attr === 1) .analyze val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where('a + 'b === 1) - .select('a + 'b as 'e) + .where("a".attr + "b".attr === 1) + .select("a".attr + "b".attr as "e") .analyze comparePlans(optimized, correctAnswer) @@ -166,15 +166,15 @@ class FilterPushdownSuite extends PlanTest { test("nondeterministic: can always push down filter through project with deterministic field") { val originalQuery = testRelation - .select('a) - .where(Rand(10) > 5 || 'a > 5) + .select("a".attr) + .where(Rand(10) > 5 || "a".attr > 5) .analyze val optimized = Optimize.execute(originalQuery) val correctAnswer = testRelation - .where(Rand(10) > 5 || 'a > 5) - .select('a) + .where(Rand(10) > 5 || "a".attr > 5) + .select("a".attr) .analyze comparePlans(optimized, correctAnswer) @@ -182,8 +182,8 @@ class FilterPushdownSuite extends PlanTest { test("nondeterministic: can't push down filter through project with nondeterministic field") { val originalQuery = testRelation - .select(Rand(10).as('rand), 'a) - .where('a > 5) + .select(Rand(10).as("rand"), "a".attr) + .where("a".attr > 5) .analyze val optimized = Optimize.execute(originalQuery) @@ -193,8 +193,8 @@ class FilterPushdownSuite extends PlanTest { test("nondeterministic: can't push down filter through aggregate with nondeterministic field") { val originalQuery = testRelation - .groupBy('a)('a, Rand(10).as('rand)) - .where('a > 5) + .groupBy("a".attr)("a".attr, Rand(10).as("rand")) + .where("a".attr > 5) .analyze val optimized = Optimize.execute(originalQuery) @@ -204,15 +204,15 @@ class FilterPushdownSuite extends PlanTest { test("nondeterministic: push down part of filter through aggregate with deterministic field") { val originalQuery = testRelation - .groupBy('a)('a) - .where('a > 5 && Rand(10) > 5) + .groupBy("a".attr)("a".attr) + .where("a".attr > 5 && Rand(10) > 5) .analyze val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where('a > 5) - .groupBy('a)('a) + .where("a".attr > 5) + .groupBy("a".attr)("a".attr) .where(Rand(10) > 5) .analyze @@ -221,22 +221,22 @@ class FilterPushdownSuite extends PlanTest { test("filters: combines filters") { val originalQuery = testRelation - .select('a) - .where('a === 1) - .where('a === 2) + .select("a".attr) + .where("a".attr === 1) + .where("a".attr === 2) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where('a === 1 && 'a === 2) - .select('a).analyze + .where("a".attr === 1 && "a".attr === 2) + .select("a".attr).analyze comparePlans(optimized, correctAnswer) } test("joins: push to either side") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = { x.join(y) @@ -245,8 +245,8 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('b === 1) - val right = testRelation.where('b === 2) + val left = testRelation.where("b".attr === 1) + val right = testRelation.where("b".attr === 2) val correctAnswer = left.join(right).analyze @@ -254,8 +254,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: push to one side") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = { x.join(y) @@ -263,7 +263,7 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('b === 1) + val left = testRelation.where("b".attr === 1) val right = testRelation val correctAnswer = left.join(right).analyze @@ -272,8 +272,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: do not push down non-deterministic filters into join condition") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") val originalQuery = x.join(y).where(Rand(10) > 5.0).analyze val optimized = Optimize.execute(originalQuery) @@ -282,8 +282,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: push to one side after transformCondition") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") val originalQuery = { x.join(y) @@ -292,7 +292,7 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('a === 1) + val left = testRelation.where("a".attr === 1) val right = testRelation1 val correctAnswer = left.join(right, condition = Some("d".attr === "b".attr || "d".attr === "c".attr)).analyze @@ -301,8 +301,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: rewrite filter to push to either side") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = { x.join(y) @@ -310,8 +310,8 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('b === 1) - val right = testRelation.where('b === 2) + val left = testRelation.where("b".attr === 1) + val right = testRelation.where("b".attr === 2) val correctAnswer = left.join(right).analyze @@ -319,16 +319,16 @@ class FilterPushdownSuite extends PlanTest { } test("joins: push down left semi join") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") val originalQuery = { x.join(y, LeftSemi, Option("x.a".attr === "y.d".attr && "x.b".attr >= 1 && "y.d".attr >= 2)) } val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('b >= 1) - val right = testRelation1.where('d >= 2) + val left = testRelation.where("b".attr >= 1) + val right = testRelation1.where("d".attr >= 2) val correctAnswer = left.join(right, LeftSemi, Option("a".attr === "d".attr)).analyze @@ -336,8 +336,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: push down left outer join #1") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = { x.join(y, LeftOuter) @@ -345,7 +345,7 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('b === 1) + val left = testRelation.where("b".attr === 1) val correctAnswer = left.join(y, LeftOuter).where("y.b".attr === 2).analyze @@ -353,8 +353,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: push down right outer join #1") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = { x.join(y, RightOuter) @@ -362,7 +362,7 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val right = testRelation.where('b === 2).subquery('d) + val right = testRelation.where("b".attr === 2).subquery("d") val correctAnswer = x.join(right, RightOuter).where("x.b".attr === 1).analyze @@ -370,8 +370,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: push down left outer join #2") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = { x.join(y, LeftOuter, Some("x.b".attr === 1)) @@ -379,7 +379,7 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('b === 2).subquery('d) + val left = testRelation.where("b".attr === 2).subquery("d") val correctAnswer = left.join(y, LeftOuter, Some("d.b".attr === 1)).where("y.b".attr === 2).analyze @@ -387,8 +387,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: push down right outer join #2") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = { x.join(y, RightOuter, Some("y.b".attr === 1)) @@ -396,7 +396,7 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val right = testRelation.where('b === 2).subquery('d) + val right = testRelation.where("b".attr === 2).subquery("d") val correctAnswer = x.join(right, RightOuter, Some("d.b".attr === 1)).where("x.b".attr === 2).analyze @@ -404,8 +404,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: push down left outer join #3") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = { x.join(y, LeftOuter, Some("y.b".attr === 1)) @@ -413,8 +413,8 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('b === 2).subquery('l) - val right = testRelation.where('b === 1).subquery('r) + val left = testRelation.where("b".attr === 2).subquery("l") + val right = testRelation.where("b".attr === 1).subquery("r") val correctAnswer = left.join(right, LeftOuter).where("r.b".attr === 2).analyze @@ -422,8 +422,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: push down right outer join #3") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = { x.join(y, RightOuter, Some("y.b".attr === 1)) @@ -431,7 +431,7 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val right = testRelation.where('b === 2).subquery('r) + val right = testRelation.where("b".attr === 2).subquery("r") val correctAnswer = x.join(right, RightOuter, Some("r.b".attr === 1)).where("x.b".attr === 2).analyze @@ -439,8 +439,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: push down left outer join #4") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = { x.join(y, LeftOuter, Some("y.b".attr === 1)) @@ -448,8 +448,8 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('b === 2).subquery('l) - val right = testRelation.where('b === 1).subquery('r) + val left = testRelation.where("b".attr === 2).subquery("l") + val right = testRelation.where("b".attr === 1).subquery("r") val correctAnswer = left.join(right, LeftOuter).where("r.b".attr === 2 && "l.c".attr === "r.c".attr).analyze @@ -457,8 +457,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: push down right outer join #4") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = { x.join(y, RightOuter, Some("y.b".attr === 1)) @@ -466,8 +466,8 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.subquery('l) - val right = testRelation.where('b === 2).subquery('r) + val left = testRelation.subquery("l") + val right = testRelation.where("b".attr === 2).subquery("r") val correctAnswer = left.join(right, RightOuter, Some("r.b".attr === 1)). where("l.b".attr === 2 && "l.c".attr === "r.c".attr).analyze @@ -476,8 +476,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: push down left outer join #5") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = { x.join(y, LeftOuter, Some("y.b".attr === 1 && "x.a".attr === 3)) @@ -485,18 +485,18 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('b === 2).subquery('l) - val right = testRelation.where('b === 1).subquery('r) + val left = testRelation.where("b".attr === 2).subquery("l") + val right = testRelation.where("b".attr === 1).subquery("r") val correctAnswer = - left.join(right, LeftOuter, Some("l.a".attr===3)). + left.join(right, LeftOuter, Some("l.a".attr === 3)). where("r.b".attr === 2 && "l.c".attr === "r.c".attr).analyze comparePlans(optimized, correctAnswer) } test("joins: push down right outer join #5") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = { x.join(y, RightOuter, Some("y.b".attr === 1 && "x.a".attr === 3)) @@ -504,8 +504,8 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('a === 3).subquery('l) - val right = testRelation.where('b === 2).subquery('r) + val left = testRelation.where("a".attr === 3).subquery("l") + val right = testRelation.where("b".attr === 2).subquery("r") val correctAnswer = left.join(right, RightOuter, Some("r.b".attr === 1)). where("l.b".attr === 2 && "l.c".attr === "r.c".attr).analyze @@ -514,8 +514,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: can't push down") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = { x.join(y, condition = Some("x.b".attr === "y.b".attr)) @@ -526,8 +526,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: conjunctive predicates") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = { x.join(y) @@ -535,8 +535,8 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('a === 1).subquery('x) - val right = testRelation.where('a === 1).subquery('y) + val left = testRelation.where("a".attr === 1).subquery("x") + val right = testRelation.where("a".attr === 1).subquery("y") val correctAnswer = left.join(right, condition = Some("x.b".attr === "y.b".attr)) .analyze @@ -545,8 +545,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: conjunctive predicates #2") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = { x.join(y) @@ -554,8 +554,8 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('a === 1).subquery('x) - val right = testRelation.subquery('y) + val left = testRelation.where("a".attr === 1).subquery("x") + val right = testRelation.subquery("y") val correctAnswer = left.join(right, condition = Some("x.b".attr === "y.b".attr)) .analyze @@ -564,9 +564,9 @@ class FilterPushdownSuite extends PlanTest { } test("joins: conjunctive predicates #3") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) - val z = testRelation.subquery('z) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") + val z = testRelation.subquery("z") val originalQuery = { z.join(x.join(y)) @@ -575,9 +575,9 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize.execute(originalQuery.analyze) - val lleft = testRelation.where('a >= 3).subquery('z) - val left = testRelation.where('a === 1).subquery('x) - val right = testRelation.subquery('y) + val lleft = testRelation.where("a".attr >= 3).subquery("z") + val left = testRelation.where("a".attr === 1).subquery("x") + val right = testRelation.subquery("y") val correctAnswer = lleft.join( left.join(right, condition = Some("x.b".attr === "y.b".attr)), @@ -588,8 +588,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: push down where clause into left anti join") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = x.join(y, LeftAnti, Some("x.b".attr === "y.b".attr)) .where("x.a".attr > 10) @@ -603,8 +603,8 @@ class FilterPushdownSuite extends PlanTest { } test("joins: only push down join conditions to the right of a left anti join") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = x.join(y, LeftAnti, @@ -620,9 +620,9 @@ class FilterPushdownSuite extends PlanTest { } test("joins: only push down join conditions to the right of an existence join") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) - val fillerVal = 'val.boolean + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") + val fillerVal = "val".attr.boolean val originalQuery = x.join(y, ExistenceJoin(fillerVal), @@ -637,19 +637,20 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } - val testRelationWithArrayType = LocalRelation('a.int, 'b.int, 'c_arr.array(IntegerType)) + val testRelationWithArrayType = + LocalRelation("a".attr.int, "b".attr.int, "c_arr".attr.array(IntegerType)) test("generate: predicate referenced no generated column") { val originalQuery = { testRelationWithArrayType - .generate(Explode('c_arr), alias = Some("arr")) - .where(('b >= 5) && ('a > 6)) + .generate(Explode("c_arr".attr), alias = Some("arr")) + .where(("b".attr >= 5) && ("a".attr > 6)) } val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = { testRelationWithArrayType - .where(('b >= 5) && ('a > 6)) - .generate(Explode('c_arr), alias = Some("arr")).analyze + .where(("b".attr >= 5) && ("a".attr > 6)) + .generate(Explode("c_arr".attr), alias = Some("arr")).analyze } comparePlans(optimized, correctAnswer) @@ -658,15 +659,15 @@ class FilterPushdownSuite extends PlanTest { test("generate: non-deterministic predicate referenced no generated column") { val originalQuery = { testRelationWithArrayType - .generate(Explode('c_arr), alias = Some("arr")) - .where(('b >= 5) && ('a + Rand(10).as("rnd") > 6) && ('col > 6)) + .generate(Explode("c_arr".attr), alias = Some("arr")) + .where(("b".attr >= 5) && ("a".attr + Rand(10).as("rnd") > 6) && ("col".attr > 6)) } val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = { testRelationWithArrayType - .where('b >= 5) - .generate(Explode('c_arr), alias = Some("arr")) - .where('a + Rand(10).as("rnd") > 6 && 'col > 6) + .where("b".attr >= 5) + .generate(Explode("c_arr".attr), alias = Some("arr")) + .where("a".attr + Rand(10).as("rnd") > 6 && "col".attr > 6) .analyze } @@ -674,18 +675,18 @@ class FilterPushdownSuite extends PlanTest { } test("generate: part of conjuncts referenced generated column") { - val generator = Explode('c_arr) + val generator = Explode("c_arr".attr) val originalQuery = { testRelationWithArrayType .generate(generator, alias = Some("arr"), outputNames = Seq("c")) - .where(('b >= 5) && ('c > 6)) + .where(("b".attr >= 5) && ("c".attr > 6)) } val optimized = Optimize.execute(originalQuery.analyze) val referenceResult = { testRelationWithArrayType - .where('b >= 5) + .where("b".attr >= 5) .generate(generator, alias = Some("arr"), outputNames = Seq("c")) - .where('c > 6).analyze + .where("c".attr > 6).analyze } // Since newly generated columns get different ids every time being analyzed @@ -705,8 +706,8 @@ class FilterPushdownSuite extends PlanTest { test("generate: all conjuncts referenced generated column") { val originalQuery = { testRelationWithArrayType - .generate(Explode('c_arr), alias = Some("arr")) - .where(('col > 6) || ('b > 5)).analyze + .generate(Explode("c_arr".attr), alias = Some("arr")) + .where(("col".attr > 6) || ("b".attr > 5)).analyze } val optimized = Optimize.execute(originalQuery) @@ -715,24 +716,24 @@ class FilterPushdownSuite extends PlanTest { test("aggregate: push down filter when filter on group by expression") { val originalQuery = testRelation - .groupBy('a)('a, count('b) as 'c) - .select('a, 'c) - .where('a === 2) + .groupBy("a".attr)("a".attr, count("b".attr) as "c") + .select("a".attr, "c".attr) + .where("a".attr === 2) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where('a === 2) - .groupBy('a)('a, count('b) as 'c) + .where("a".attr === 2) + .groupBy("a".attr)("a".attr, count("b".attr) as "c") .analyze comparePlans(optimized, correctAnswer) } test("aggregate: don't push down filter when filter not on group by expression") { val originalQuery = testRelation - .select('a, 'b) - .groupBy('a)('a, count('b) as 'c) - .where('c === 2L) + .select("a".attr, "b".attr) + .groupBy("a".attr)("a".attr, count("b".attr) as "c") + .where("c".attr === 2L) val optimized = Optimize.execute(originalQuery.analyze) @@ -741,17 +742,17 @@ class FilterPushdownSuite extends PlanTest { test("aggregate: push down filters partially which are subset of group by expressions") { val originalQuery = testRelation - .select('a, 'b) - .groupBy('a)('a, count('b) as 'c) - .where('c === 2L && 'a === 3) + .select("a".attr, "b".attr) + .groupBy("a".attr)("a".attr, count("b".attr) as "c") + .where("c".attr === 2L && "a".attr === 3) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where('a === 3) - .select('a, 'b) - .groupBy('a)('a, count('b) as 'c) - .where('c === 2L) + .where("a".attr === 3) + .select("a".attr, "b".attr) + .groupBy("a".attr)("a".attr, count("b".attr) as "c") + .where("c".attr === 2L) .analyze comparePlans(optimized, correctAnswer) @@ -759,17 +760,17 @@ class FilterPushdownSuite extends PlanTest { test("aggregate: push down filters with alias") { val originalQuery = testRelation - .select('a, 'b) - .groupBy('a)(('a + 1) as 'aa, count('b) as 'c) - .where(('c === 2L || 'aa > 4) && 'aa < 3) + .select("a".attr, "b".attr) + .groupBy("a".attr)(("a".attr + 1) as "aa", count("b".attr) as "c") + .where(("c".attr === 2L || "aa".attr > 4) && "aa".attr < 3) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where('a + 1 < 3) - .select('a, 'b) - .groupBy('a)(('a + 1) as 'aa, count('b) as 'c) - .where('c === 2L || 'aa > 4) + .where("a".attr + 1 < 3) + .select("a".attr, "b".attr) + .groupBy("a".attr)(("a".attr + 1) as "aa", count("b".attr) as "c") + .where("c".attr === 2L || "aa".attr > 4) .analyze comparePlans(optimized, correctAnswer) @@ -777,17 +778,17 @@ class FilterPushdownSuite extends PlanTest { test("aggregate: push down filters with literal") { val originalQuery = testRelation - .select('a, 'b) - .groupBy('a)('a, count('b) as 'c, "s" as 'd) - .where('c === 2L && 'd === "s") + .select("a".attr, "b".attr) + .groupBy("a".attr)("a".attr, count("b".attr) as "c", "s" as "d") + .where("c".attr === 2L && "d".attr === "s") val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where("s" === "s") - .select('a, 'b) - .groupBy('a)('a, count('b) as 'c, "s" as 'd) - .where('c === 2L) + .select("a".attr, "b".attr) + .groupBy("a".attr)("a".attr, count("b".attr) as "c", "s" as "d") + .where("c".attr === 2L) .analyze comparePlans(optimized, correctAnswer) @@ -795,16 +796,16 @@ class FilterPushdownSuite extends PlanTest { test("aggregate: don't push down filters that are nondeterministic") { val originalQuery = testRelation - .select('a, 'b) - .groupBy('a)('a + Rand(10) as 'aa, count('b) as 'c, Rand(11).as("rnd")) - .where('c === 2L && 'aa + Rand(10).as("rnd") === 3 && 'rnd === 5) + .select("a".attr, "b".attr) + .groupBy("a".attr)("a".attr + Rand(10) as "aa", count("b".attr) as "c", Rand(11).as("rnd")) + .where("c".attr === 2L && "aa".attr + Rand(10).as("rnd") === 3 && "rnd".attr === 5) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select('a, 'b) - .groupBy('a)('a + Rand(10) as 'aa, count('b) as 'c, Rand(11).as("rnd")) - .where('c === 2L && 'aa + Rand(10).as("rnd") === 3 && 'rnd === 5) + .select("a".attr, "b".attr) + .groupBy("a".attr)("a".attr + Rand(10) as "aa", count("b".attr) as "c", Rand(11).as("rnd")) + .where("c".attr === 2L && "aa".attr + Rand(10).as("rnd") === 3 && "rnd".attr === 5) .analyze comparePlans(optimized, correctAnswer) @@ -812,15 +813,15 @@ class FilterPushdownSuite extends PlanTest { test("SPARK-17712: aggregate: don't push down filters that are data-independent") { val originalQuery = LocalRelation.apply(testRelation.output, Seq.empty) - .select('a, 'b) - .groupBy('a)(count('a)) + .select("a".attr, "b".attr) + .groupBy("a".attr)(count("a".attr)) .where(false) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select('a, 'b) - .groupBy('a)(count('a)) + .select("a".attr, "b".attr) + .groupBy("a".attr)(count("a".attr)) .where(false) .analyze @@ -829,7 +830,7 @@ class FilterPushdownSuite extends PlanTest { test("aggregate: don't push filters if the aggregate has no grouping expressions") { val originalQuery = LocalRelation.apply(testRelation.output, Seq.empty) - .select('a, 'b) + .select("a".attr, "b".attr) .groupBy()(count(1)) .where(false) @@ -841,17 +842,17 @@ class FilterPushdownSuite extends PlanTest { } test("union") { - val testRelation2 = LocalRelation('d.int, 'e.int, 'f.int) + val testRelation2 = LocalRelation("d".attr.int, "e".attr.int, "f".attr.int) val originalQuery = Union(Seq(testRelation, testRelation2)) - .where('a === 2L && 'b + Rand(10).as("rnd") === 3 && 'c > 5L) + .where("a".attr === 2L && "b".attr + Rand(10).as("rnd") === 3 && "c".attr > 5L) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = Union(Seq( - testRelation.where('a === 2L && 'c > 5L), - testRelation2.where('d === 2L && 'f > 5L))) - .where('b + Rand(10).as("rnd") === 3) + testRelation.where("a".attr === 2L && "c".attr > 5L), + testRelation2.where("d".attr === 2L && "f".attr > 5L))) + .where("b".attr + Rand(10).as("rnd") === 3) .analyze comparePlans(optimized, correctAnswer) @@ -859,7 +860,7 @@ class FilterPushdownSuite extends PlanTest { test("expand") { val agg = testRelation - .groupBy(Cube(Seq('a, 'b)))('a, 'b, sum('c)) + .groupBy(Cube(Seq("a".attr, "b".attr)))("a".attr, "b".attr, sum("c".attr)) .analyze .asInstanceOf[Aggregate] @@ -873,9 +874,9 @@ class FilterPushdownSuite extends PlanTest { } test("predicate subquery: push down simple") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) - val z = LocalRelation('a.int, 'b.int, 'c.int).subquery('z) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") + val z = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int).subquery("z") val query = x .join(y, Inner, Option("x.a".attr === "y.a".attr)) @@ -890,10 +891,10 @@ class FilterPushdownSuite extends PlanTest { } test("predicate subquery: push down complex") { - val w = testRelation.subquery('w) - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) - val z = LocalRelation('a.int, 'b.int, 'c.int).subquery('z) + val w = testRelation.subquery("w") + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") + val z = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int).subquery("z") val query = w .join(x, Inner, Option("w.a".attr === "x.a".attr)) @@ -910,9 +911,9 @@ class FilterPushdownSuite extends PlanTest { } test("SPARK-20094: don't push predicate with IN subquery into join condition") { - val x = testRelation.subquery('x) - val z = testRelation.subquery('z) - val w = testRelation1.subquery('w) + val x = testRelation.subquery("x") + val z = testRelation.subquery("z") + val w = testRelation1.subquery("w") val queryPlan = x .join(z) @@ -930,66 +931,71 @@ class FilterPushdownSuite extends PlanTest { } test("Window: predicate push down -- basic") { - val winExpr = windowExpr(count('b), windowSpec('a :: Nil, 'b.asc :: Nil, UnspecifiedFrame)) + val winExpr = windowExpr(count("b".attr), + windowSpec("a".attr :: Nil, "b".attr.asc :: Nil, UnspecifiedFrame)) - val originalQuery = testRelation.select('a, 'b, 'c, winExpr.as('window)).where('a > 1) + val originalQuery = + testRelation.select("a".attr, "b".attr, "c".attr, winExpr.as("window")).where("a".attr > 1) val correctAnswer = testRelation - .where('a > 1).select('a, 'b, 'c) - .window(winExpr.as('window) :: Nil, 'a :: Nil, 'b.asc :: Nil) - .select('a, 'b, 'c, 'window).analyze + .where("a".attr > 1).select("a".attr, "b".attr, "c".attr) + .window(winExpr.as("window") :: Nil, "a".attr :: Nil, "b".attr.asc :: Nil) + .select("a".attr, "b".attr, "c".attr, "window".attr).analyze comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer) } test("Window: predicate push down -- predicates with compound predicate using only one column") { - val winExpr = - windowExpr(count('b), windowSpec('a.attr :: 'b.attr :: Nil, 'b.asc :: Nil, UnspecifiedFrame)) + val winExpr = windowExpr(count("b".attr), + windowSpec("a".attr :: "b".attr :: Nil, "b".attr.asc :: Nil, UnspecifiedFrame)) - val originalQuery = testRelation.select('a, 'b, 'c, winExpr.as('window)).where('a * 3 > 15) + val originalQuery = testRelation.select("a".attr, "b".attr, "c".attr, + winExpr.as("window")).where("a".attr * 3 > 15) val correctAnswer = testRelation - .where('a * 3 > 15).select('a, 'b, 'c) - .window(winExpr.as('window) :: Nil, 'a.attr :: 'b.attr :: Nil, 'b.asc :: Nil) - .select('a, 'b, 'c, 'window).analyze + .where("a".attr * 3 > 15).select("a".attr, "b".attr, "c".attr) + .window(winExpr.as("window") :: Nil, "a".attr :: "b".attr :: Nil, "b".attr.asc :: Nil) + .select("a".attr, "b".attr, "c".attr, "window".attr).analyze comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer) } test("Window: predicate push down -- multi window expressions with the same window spec") { - val winSpec = windowSpec('a.attr :: 'b.attr :: Nil, 'b.asc :: Nil, UnspecifiedFrame) - val winExpr1 = windowExpr(count('b), winSpec) - val winExpr2 = windowExpr(sum('b), winSpec) + val winSpec = windowSpec("a".attr :: "b".attr :: Nil, "b".attr.asc :: Nil, UnspecifiedFrame) + val winExpr1 = windowExpr(count("b".attr), winSpec) + val winExpr2 = windowExpr(sum("b".attr), winSpec) val originalQuery = testRelation - .select('a, 'b, 'c, winExpr1.as('window1), winExpr2.as('window2)).where('a > 1) + .select("a".attr, "b".attr, "c".attr, winExpr1.as("window1"), winExpr2.as("window2")) + .where("a".attr > 1) val correctAnswer = testRelation - .where('a > 1).select('a, 'b, 'c) - .window(winExpr1.as('window1) :: winExpr2.as('window2) :: Nil, - 'a.attr :: 'b.attr :: Nil, 'b.asc :: Nil) - .select('a, 'b, 'c, 'window1, 'window2).analyze + .where("a".attr > 1).select("a".attr, "b".attr, "c".attr) + .window(winExpr1.as("window1") :: winExpr2.as("window2") :: Nil, + "a".attr :: "b".attr :: Nil, "b".attr.asc :: Nil) + .select("a".attr, "b".attr, "c".attr, "window1".attr, "window2".attr).analyze comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer) } test("Window: predicate push down -- multi window specification - 1") { // order by clauses are different between winSpec1 and winSpec2 - val winSpec1 = windowSpec('a.attr :: 'b.attr :: Nil, 'b.asc :: Nil, UnspecifiedFrame) - val winExpr1 = windowExpr(count('b), winSpec1) - val winSpec2 = windowSpec('a.attr :: 'b.attr :: Nil, 'a.asc :: Nil, UnspecifiedFrame) - val winExpr2 = windowExpr(count('b), winSpec2) + val winSpec1 = windowSpec("a".attr :: "b".attr :: Nil, "b".attr.asc :: Nil, UnspecifiedFrame) + val winExpr1 = windowExpr(count("b".attr), winSpec1) + val winSpec2 = windowSpec("a".attr :: "b".attr :: Nil, "a".attr.asc :: Nil, UnspecifiedFrame) + val winExpr2 = windowExpr(count("b".attr), winSpec2) val originalQuery = testRelation - .select('a, 'b, 'c, winExpr1.as('window1), winExpr2.as('window2)).where('a > 1) + .select("a".attr, "b".attr, "c".attr, winExpr1.as("window1"), winExpr2.as("window2")) + .where("a".attr > 1) val correctAnswer1 = testRelation - .where('a > 1).select('a, 'b, 'c) - .window(winExpr1.as('window1) :: Nil, 'a.attr :: 'b.attr :: Nil, 'b.asc :: Nil) - .window(winExpr2.as('window2) :: Nil, 'a.attr :: 'b.attr :: Nil, 'a.asc :: Nil) - .select('a, 'b, 'c, 'window1, 'window2).analyze + .where("a".attr > 1).select("a".attr, "b".attr, "c".attr) + .window(winExpr1.as("window1") :: Nil, "a".attr :: "b".attr :: Nil, "b".attr.asc :: Nil) + .window(winExpr2.as("window2") :: Nil, "a".attr :: "b".attr :: Nil, "a".attr.asc :: Nil) + .select("a".attr, "b".attr, "c".attr, "window1".attr, "window2".attr).analyze val correctAnswer2 = testRelation - .where('a > 1).select('a, 'b, 'c) - .window(winExpr2.as('window2) :: Nil, 'a.attr :: 'b.attr :: Nil, 'a.asc :: Nil) - .window(winExpr1.as('window1) :: Nil, 'a.attr :: 'b.attr :: Nil, 'b.asc :: Nil) - .select('a, 'b, 'c, 'window1, 'window2).analyze + .where("a".attr > 1).select("a".attr, "b".attr, "c".attr) + .window(winExpr2.as("window2") :: Nil, "a".attr :: "b".attr :: Nil, "a".attr.asc :: Nil) + .window(winExpr1.as("window1") :: Nil, "a".attr :: "b".attr :: Nil, "b".attr.asc :: Nil) + .select("a".attr, "b".attr, "c".attr, "window1".attr, "window2".attr).analyze // When Analyzer adding Window operators after grouping the extracted Window Expressions // based on their Partition and Order Specs, the order of Window operators is @@ -1004,24 +1010,25 @@ class FilterPushdownSuite extends PlanTest { test("Window: predicate push down -- multi window specification - 2") { // partitioning clauses are different between winSpec1 and winSpec2 - val winSpec1 = windowSpec('a.attr :: Nil, 'b.asc :: Nil, UnspecifiedFrame) - val winExpr1 = windowExpr(count('b), winSpec1) - val winSpec2 = windowSpec('b.attr :: Nil, 'b.asc :: Nil, UnspecifiedFrame) - val winExpr2 = windowExpr(count('a), winSpec2) + val winSpec1 = windowSpec("a".attr :: Nil, "b".attr.asc :: Nil, UnspecifiedFrame) + val winExpr1 = windowExpr(count("b".attr), winSpec1) + val winSpec2 = windowSpec("b".attr :: Nil, "b".attr.asc :: Nil, UnspecifiedFrame) + val winExpr2 = windowExpr(count("a".attr), winSpec2) val originalQuery = testRelation - .select('a, winExpr1.as('window1), 'b, 'c, winExpr2.as('window2)).where('b > 1) + .select("a".attr, winExpr1.as("window1"), "b".attr, "c".attr, winExpr2.as("window2")) + .where("b".attr > 1) - val correctAnswer1 = testRelation.select('a, 'b, 'c) - .window(winExpr1.as('window1) :: Nil, 'a.attr :: Nil, 'b.asc :: Nil) - .where('b > 1) - .window(winExpr2.as('window2) :: Nil, 'b.attr :: Nil, 'b.asc :: Nil) - .select('a, 'window1, 'b, 'c, 'window2).analyze + val correctAnswer1 = testRelation.select("a".attr, "b".attr, "c".attr) + .window(winExpr1.as("window1") :: Nil, "a".attr :: Nil, "b".attr.asc :: Nil) + .where("b".attr > 1) + .window(winExpr2.as("window2") :: Nil, "b".attr :: Nil, "b".attr.asc :: Nil) + .select("a".attr, "window1".attr, "b".attr, "c".attr, "window2".attr).analyze - val correctAnswer2 = testRelation.select('a, 'b, 'c) - .window(winExpr2.as('window2) :: Nil, 'b.attr :: Nil, 'b.asc :: Nil) - .window(winExpr1.as('window1) :: Nil, 'a.attr :: Nil, 'b.asc :: Nil) - .where('b > 1) - .select('a, 'window1, 'b, 'c, 'window2).analyze + val correctAnswer2 = testRelation.select("a".attr, "b".attr, "c".attr) + .window(winExpr2.as("window2") :: Nil, "b".attr :: Nil, "b".attr.asc :: Nil) + .window(winExpr1.as("window1") :: Nil, "a".attr :: Nil, "b".attr.asc :: Nil) + .where("b".attr > 1) + .select("a".attr, "window1".attr, "b".attr, "c".attr, "window2".attr).analyze val optimizedQuery = Optimize.execute(originalQuery.analyze) // When Analyzer adding Window operators after grouping the extracted Window Expressions @@ -1036,13 +1043,16 @@ class FilterPushdownSuite extends PlanTest { test("Window: predicate push down -- predicates with multiple partitioning columns") { val winExpr = - windowExpr(count('b), windowSpec('a.attr :: 'b.attr :: Nil, 'b.asc :: Nil, UnspecifiedFrame)) + windowExpr(count("b".attr), + windowSpec("a".attr :: "b".attr :: Nil, "b".attr.asc :: Nil, UnspecifiedFrame)) - val originalQuery = testRelation.select('a, 'b, 'c, winExpr.as('window)).where('a + 'b > 1) + val originalQuery = testRelation + .select("a".attr, "b".attr, "c".attr, winExpr.as("window")) + .where("a".attr + "b".attr > 1) val correctAnswer = testRelation - .where('a + 'b > 1).select('a, 'b, 'c) - .window(winExpr.as('window) :: Nil, 'a.attr :: 'b.attr :: Nil, 'b.asc :: Nil) - .select('a, 'b, 'c, 'window).analyze + .where("a".attr + "b".attr > 1).select("a".attr, "b".attr, "c".attr) + .window(winExpr.as("window") :: Nil, "a".attr :: "b".attr :: Nil, "b".attr.asc :: Nil) + .select("a".attr, "b".attr, "c".attr, "window".attr).analyze comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer) } @@ -1052,75 +1062,85 @@ class FilterPushdownSuite extends PlanTest { // to the alias that is defined as the same expression ignore("Window: predicate push down -- complex predicate with the same expressions") { val winSpec = windowSpec( - partitionSpec = 'a.attr + 'b.attr :: Nil, - orderSpec = 'b.asc :: Nil, + partitionSpec = "a".attr + "b".attr :: Nil, + orderSpec = "b".attr.asc :: Nil, UnspecifiedFrame) - val winExpr = windowExpr(count('b), winSpec) + val winExpr = windowExpr(count("b".attr), winSpec) val winSpecAnalyzed = windowSpec( - partitionSpec = '_w0.attr :: Nil, - orderSpec = 'b.asc :: Nil, + partitionSpec = "_w0".attr :: Nil, + orderSpec = "b".attr.asc :: Nil, UnspecifiedFrame) - val winExprAnalyzed = windowExpr(count('b), winSpecAnalyzed) + val winExprAnalyzed = windowExpr(count("b".attr), winSpecAnalyzed) - val originalQuery = testRelation.select('a, 'b, 'c, winExpr.as('window)).where('a + 'b > 1) + val originalQuery = testRelation + .select("a".attr, "b".attr, "c".attr, winExpr.as("window")) + .where("a".attr + "b".attr > 1) val correctAnswer = testRelation - .where('a + 'b > 1).select('a, 'b, 'c, ('a + 'b).as("_w0")) - .window(winExprAnalyzed.as('window) :: Nil, '_w0 :: Nil, 'b.asc :: Nil) - .select('a, 'b, 'c, 'window).analyze + .where("a".attr + "b".attr > 1) + .select("a".attr, "b".attr, "c".attr, ("a".attr + "b".attr).as("_w0")) + .window(winExprAnalyzed.as("window") :: Nil, "_w0".attr :: Nil, "b".attr.asc :: Nil) + .select("a".attr, "b".attr, "c".attr, "window".attr).analyze comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer) } test("Window: no predicate push down -- predicates are not from partitioning keys") { val winSpec = windowSpec( - partitionSpec = 'a.attr :: 'b.attr :: Nil, - orderSpec = 'b.asc :: Nil, + partitionSpec = "a".attr :: "b".attr :: Nil, + orderSpec = "b".attr.asc :: Nil, UnspecifiedFrame) - val winExpr = windowExpr(count('b), winSpec) + val winExpr = windowExpr(count("b".attr), winSpec) // No push down: the predicate is c > 1, but the partitioning key is (a, b). - val originalQuery = testRelation.select('a, 'b, 'c, winExpr.as('window)).where('c > 1) - val correctAnswer = testRelation.select('a, 'b, 'c) - .window(winExpr.as('window) :: Nil, 'a.attr :: 'b.attr :: Nil, 'b.asc :: Nil) - .where('c > 1).select('a, 'b, 'c, 'window).analyze + val originalQuery = testRelation + .select("a".attr, "b".attr, "c".attr, winExpr.as("window")) + .where("c".attr > 1) + val correctAnswer = testRelation.select("a".attr, "b".attr, "c".attr) + .window(winExpr.as("window") :: Nil, "a".attr :: "b".attr :: Nil, "b".attr.asc :: Nil) + .where("c".attr > 1).select("a".attr, "b".attr, "c".attr, "window".attr).analyze comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer) } test("Window: no predicate push down -- partial compound partition key") { val winSpec = windowSpec( - partitionSpec = 'a.attr + 'b.attr :: 'b.attr :: Nil, - orderSpec = 'b.asc :: Nil, + partitionSpec = "a".attr + "b".attr :: "b".attr :: Nil, + orderSpec = "b".attr.asc :: Nil, UnspecifiedFrame) - val winExpr = windowExpr(count('b), winSpec) + val winExpr = windowExpr(count("b".attr), winSpec) // No push down: the predicate is a > 1, but the partitioning key is (a + b, b) - val originalQuery = testRelation.select('a, 'b, 'c, winExpr.as('window)).where('a > 1) + val originalQuery = + testRelation.select("a".attr, "b".attr, "c".attr, winExpr.as("window")).where("a".attr > 1) val winSpecAnalyzed = windowSpec( - partitionSpec = '_w0.attr :: 'b.attr :: Nil, - orderSpec = 'b.asc :: Nil, + partitionSpec = "_w0".attr :: "b".attr :: Nil, + orderSpec = "b".attr.asc :: Nil, UnspecifiedFrame) - val winExprAnalyzed = windowExpr(count('b), winSpecAnalyzed) - val correctAnswer = testRelation.select('a, 'b, 'c, ('a + 'b).as("_w0")) - .window(winExprAnalyzed.as('window) :: Nil, '_w0 :: 'b.attr :: Nil, 'b.asc :: Nil) - .where('a > 1).select('a, 'b, 'c, 'window).analyze + val winExprAnalyzed = windowExpr(count("b".attr), winSpecAnalyzed) + val correctAnswer = testRelation + .select("a".attr, "b".attr, "c".attr, ("a".attr + "b".attr).as("_w0")) + .window( + winExprAnalyzed.as("window") :: Nil, "_w0".attr :: "b".attr :: Nil, "b".attr.asc :: Nil) + .where("a".attr > 1).select("a".attr, "b".attr, "c".attr, "window".attr).analyze comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer) } test("Window: no predicate push down -- complex predicates containing non partitioning columns") { val winSpec = - windowSpec(partitionSpec = 'b.attr :: Nil, orderSpec = 'b.asc :: Nil, UnspecifiedFrame) - val winExpr = windowExpr(count('b), winSpec) + windowSpec(partitionSpec = "b".attr :: Nil, orderSpec = "b".attr.asc :: Nil, UnspecifiedFrame) + val winExpr = windowExpr(count("b".attr), winSpec) // No push down: the predicate is a + b > 1, but the partitioning key is b. - val originalQuery = testRelation.select('a, 'b, 'c, winExpr.as('window)).where('a + 'b > 1) + val originalQuery = testRelation + .select("a".attr, "b".attr, "c".attr, winExpr.as("window")) + .where("a".attr + "b".attr > 1) val correctAnswer = testRelation - .select('a, 'b, 'c) - .window(winExpr.as('window) :: Nil, 'b.attr :: Nil, 'b.asc :: Nil) - .where('a + 'b > 1).select('a, 'b, 'c, 'window).analyze + .select("a".attr, "b".attr, "c".attr) + .window(winExpr.as("window") :: Nil, "b".attr :: Nil, "b".attr.asc :: Nil) + .where("a".attr + "b".attr > 1).select("a".attr, "b".attr, "c".attr, "window".attr).analyze comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer) } @@ -1128,52 +1148,55 @@ class FilterPushdownSuite extends PlanTest { // complex predicates with the same references but different expressions test("Window: no predicate push down -- complex predicate with different expressions") { val winSpec = windowSpec( - partitionSpec = 'a.attr + 'b.attr :: Nil, - orderSpec = 'b.asc :: Nil, + partitionSpec = "a".attr + "b".attr :: Nil, + orderSpec = "b".attr.asc :: Nil, UnspecifiedFrame) - val winExpr = windowExpr(count('b), winSpec) + val winExpr = windowExpr(count("b".attr), winSpec) val winSpecAnalyzed = windowSpec( - partitionSpec = '_w0.attr :: Nil, - orderSpec = 'b.asc :: Nil, + partitionSpec = "_w0".attr :: Nil, + orderSpec = "b".attr.asc :: Nil, UnspecifiedFrame) - val winExprAnalyzed = windowExpr(count('b), winSpecAnalyzed) + val winExprAnalyzed = windowExpr(count("b".attr), winSpecAnalyzed) // No push down: the predicate is a + b > 1, but the partitioning key is a + b. - val originalQuery = testRelation.select('a, 'b, 'c, winExpr.as('window)).where('a - 'b > 1) - val correctAnswer = testRelation.select('a, 'b, 'c, ('a + 'b).as("_w0")) - .window(winExprAnalyzed.as('window) :: Nil, '_w0 :: Nil, 'b.asc :: Nil) - .where('a - 'b > 1).select('a, 'b, 'c, 'window).analyze + val originalQuery = testRelation + .select("a".attr, "b".attr, "c".attr, winExpr.as("window")) + .where("a".attr - "b".attr > 1) + val correctAnswer = testRelation + .select("a".attr, "b".attr, "c".attr, ("a".attr + "b".attr).as("_w0")) + .window(winExprAnalyzed.as("window") :: Nil, "_w0".attr :: Nil, "b".attr.asc :: Nil) + .where("a".attr - "b".attr > 1).select("a".attr, "b".attr, "c".attr, "window".attr).analyze comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer) } test("watermark pushdown: no pushdown on watermark attribute #1") { val interval = new CalendarInterval(2, 2, 2000L) - val relation = LocalRelation(attrA, 'b.timestamp, attrC) + val relation = LocalRelation(attrA, "b".attr.timestamp, attrC) // Verify that all conditions except the watermark touching condition are pushed down // by the optimizer and others are not. - val originalQuery = EventTimeWatermark('b, interval, relation) - .where('a === 5 && 'b === new java.sql.Timestamp(0) && 'c === 5) + val originalQuery = EventTimeWatermark("b".attr, interval, relation) + .where("a".attr === 5 && "b".attr === new java.sql.Timestamp(0) && "c".attr === 5) val correctAnswer = EventTimeWatermark( - 'b, interval, relation.where('a === 5 && 'c === 5)) - .where('b === new java.sql.Timestamp(0)) + "b".attr, interval, relation.where("a".attr === 5 && "c".attr === 5)) + .where("b".attr === new java.sql.Timestamp(0)) comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) } test("watermark pushdown: no pushdown for nondeterministic filter") { val interval = new CalendarInterval(2, 2, 2000L) - val relation = LocalRelation(attrA, attrB, 'c.timestamp) + val relation = LocalRelation(attrA, attrB, "c".attr.timestamp) // Verify that all conditions except the watermark touching condition are pushed down // by the optimizer and others are not. - val originalQuery = EventTimeWatermark('c, interval, relation) - .where('a === 5 && 'b === Rand(10) && 'c === new java.sql.Timestamp(0)) + val originalQuery = EventTimeWatermark("c".attr, interval, relation) + .where("a".attr === 5 && "b".attr === Rand(10) && "c".attr === new java.sql.Timestamp(0)) val correctAnswer = EventTimeWatermark( - 'c, interval, relation.where('a === 5)) - .where('b === Rand(10) && 'c === new java.sql.Timestamp(0)) + "c".attr, interval, relation.where("a".attr === 5)) + .where("b".attr === Rand(10) && "c".attr === new java.sql.Timestamp(0)) comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze, checkAnalysis = false) @@ -1181,14 +1204,14 @@ class FilterPushdownSuite extends PlanTest { test("watermark pushdown: full pushdown") { val interval = new CalendarInterval(2, 2, 2000L) - val relation = LocalRelation(attrA, attrB, 'c.timestamp) + val relation = LocalRelation(attrA, attrB, "c".attr.timestamp) // Verify that all conditions except the watermark touching condition are pushed down // by the optimizer and others are not. - val originalQuery = EventTimeWatermark('c, interval, relation) - .where('a === 5 && 'b === 10) + val originalQuery = EventTimeWatermark("c".attr, interval, relation) + .where("a".attr === 5 && "b".attr === 10) val correctAnswer = EventTimeWatermark( - 'c, interval, relation.where('a === 5 && 'b === 10)) + "c".attr, interval, relation.where("a".attr === 5 && "b".attr === 10)) comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze, checkAnalysis = false) @@ -1196,12 +1219,12 @@ class FilterPushdownSuite extends PlanTest { test("watermark pushdown: no pushdown on watermark attribute #2") { val interval = new CalendarInterval(2, 2, 2000L) - val relation = LocalRelation('a.timestamp, attrB, attrC) + val relation = LocalRelation("a".attr.timestamp, attrB, attrC) - val originalQuery = EventTimeWatermark('a, interval, relation) - .where('a === new java.sql.Timestamp(0) && 'b === 10) - val correctAnswer = EventTimeWatermark( - 'a, interval, relation.where('b === 10)).where('a === new java.sql.Timestamp(0)) + val originalQuery = EventTimeWatermark("a".attr, interval, relation) + .where("a".attr === new java.sql.Timestamp(0) && "b".attr === 10) + val correctAnswer = EventTimeWatermark("a".attr, interval, + relation.where("b".attr === 10)).where("a".attr === new java.sql.Timestamp(0)) comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze, checkAnalysis = false) @@ -1209,22 +1232,22 @@ class FilterPushdownSuite extends PlanTest { test("push down predicate through expand") { val query = - Filter('a > 1, + Filter("a".attr > 1, Expand( Seq( - Seq('a, 'b, 'c, Literal.create(null, StringType), 1), - Seq('a, 'b, 'c, 'a, 2)), - Seq('a, 'b, 'c), + Seq("a".attr, "b".attr, "c".attr, Literal.create(null, StringType), 1), + Seq("a".attr, "b".attr, "c".attr, "a".attr, 2)), + Seq("a".attr, "b".attr, "c".attr), testRelation)).analyze val optimized = Optimize.execute(query) val expected = Expand( Seq( - Seq('a, 'b, 'c, Literal.create(null, StringType), 1), - Seq('a, 'b, 'c, 'a, 2)), - Seq('a, 'b, 'c), - Filter('a > 1, testRelation)).analyze + Seq("a".attr, "b".attr, "c".attr, Literal.create(null, StringType), 1), + Seq("a".attr, "b".attr, "c".attr, "a".attr, 2)), + Seq("a".attr, "b".attr, "c".attr), + Filter("a".attr > 1, testRelation)).analyze comparePlans(optimized, expected) } @@ -1252,8 +1275,8 @@ class FilterPushdownSuite extends PlanTest { } test("push down filter predicates through inner join") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = x.join(y).where(("x.b".attr === "y.b".attr) && (simpleDisjunctivePredicate)) @@ -1262,8 +1285,8 @@ class FilterPushdownSuite extends PlanTest { } test("push down join predicates through inner join") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = x.join(y, condition = Some(("x.b".attr === "y.b".attr) && (simpleDisjunctivePredicate))) @@ -1273,8 +1296,8 @@ class FilterPushdownSuite extends PlanTest { } test("push down complex predicates through inner join") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val joinCondition = (("x.b".attr === "y.b".attr) && ((("x.a".attr === 5) && ("y.a".attr >= 2) && ("y.a".attr <= 3)) @@ -1284,17 +1307,18 @@ class FilterPushdownSuite extends PlanTest { val originalQuery = x.join(y, condition = Some(joinCondition)) val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where( - ('a === 5 || 'a === 2 || 'a === 1)).subquery('x) + ("a".attr === 5 || "a".attr === 2 || "a".attr === 1)).subquery("x") val right = testRelation.where( - ('a >= 2 && 'a <= 3) || ('a >= 1 && 'a <= 14) || ('a >= 9 && 'a <= 27)).subquery('y) + ("a".attr >= 2 && "a".attr <= 3) || ("a".attr >= 1 && "a".attr <= 14) || + ("a".attr >= 9 && "a".attr <= 27)).subquery("y") val correctAnswer = left.join(right, condition = Some(joinCondition)).analyze comparePlans(optimized, correctAnswer) } test("push down predicates(with NOT predicate) through inner join") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = x.join(y, condition = Some(("x.b".attr === "y.b".attr) @@ -1302,8 +1326,8 @@ class FilterPushdownSuite extends PlanTest { && ("x.a".attr < 2 || ("y.a".attr > 13)) || ("x.a".attr > 1) && ("y.a".attr > 11)))) val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('a <= 3 || 'a >= 2).subquery('x) - val right = testRelation.subquery('y) + val left = testRelation.where("a".attr <= 3 || "a".attr >= 2).subquery("x") + val right = testRelation.subquery("y") val correctAnswer = left.join(right, condition = Some("x.b".attr === "y.b".attr && (("x.a".attr <= 3) || (("x.a".attr >= 2) && ("y.a".attr <= 13))) @@ -1313,16 +1337,16 @@ class FilterPushdownSuite extends PlanTest { } test("push down predicates through left join") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = x.join(y, joinType = LeftOuter, condition = Some(("x.b".attr === "y.b".attr) && simpleDisjunctivePredicate)) val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.subquery('x) - val right = testRelation.where('a > 13 || 'a > 11).subquery('y) + val left = testRelation.subquery("x") + val right = testRelation.where("a".attr > 13 || "a".attr > 11).subquery("y") val correctAnswer = left.join(right, joinType = LeftOuter, condition = Some("x.b".attr === "y.b".attr && (("x.a".attr > 3) && ("y.a".attr > 13) || ("x.a".attr > 1) && ("y.a".attr > 11)))) @@ -1332,16 +1356,16 @@ class FilterPushdownSuite extends PlanTest { } test("push down predicates through right join") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = x.join(y, joinType = RightOuter, condition = Some(("x.b".attr === "y.b".attr) && simpleDisjunctivePredicate)) val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('a > 3 || 'a > 1).subquery('x) - val right = testRelation.subquery('y) + val left = testRelation.where("a".attr > 3 || "a".attr > 1).subquery("x") + val right = testRelation.subquery("y") val correctAnswer = left.join(right, joinType = RightOuter, condition = Some("x.b".attr === "y.b".attr && (("x.a".attr > 3) && ("y.a".attr > 13) || ("x.a".attr > 1) && ("y.a".attr > 11)))) @@ -1351,16 +1375,16 @@ class FilterPushdownSuite extends PlanTest { } test("SPARK-32302: avoid generating too many predicates") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = x.join(y, condition = Some(("x.b".attr === "y.b".attr) && ((("x.a".attr > 3) && ("x.a".attr < 13) && ("y.c".attr <= 5)) || (("y.a".attr > 2) && ("y.c".attr < 1))))) val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.subquery('x) - val right = testRelation.where('c <= 5 || ('a > 2 && 'c < 1)).subquery('y) + val left = testRelation.subquery("x") + val right = testRelation.where("c".attr <= 5 || ("a".attr > 2 && "c".attr < 1)).subquery("y") val correctAnswer = left.join(right, condition = Some("x.b".attr === "y.b".attr && ((("x.a".attr > 3) && ("x.a".attr < 13) && ("y.c".attr <= 5)) || (("y.a".attr > 2) && ("y.c".attr < 1))))).analyze @@ -1369,16 +1393,16 @@ class FilterPushdownSuite extends PlanTest { } test("push down predicate through multiple joins") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) - val z = testRelation.subquery('z) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") + val z = testRelation.subquery("z") val xJoinY = x.join(y, condition = Some("x.b".attr === "y.b".attr)) val originalQuery = z.join(xJoinY, condition = Some("x.a".attr === "z.a".attr && simpleDisjunctivePredicate)) val optimized = Optimize.execute(originalQuery.analyze) - val left = x.where('a > 3 || 'a > 1) - val right = y.where('a > 13 || 'a > 11) + val left = x.where("a".attr > 3 || "a".attr > 1) + val right = y.where("a".attr > 13 || "a".attr > 11) val correctAnswer = z.join(left.join(right, condition = Some("x.b".attr === "y.b".attr && simpleDisjunctivePredicate)), condition = Some("x.a".attr === "z.a".attr)).analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala index 92e4fa345e2a..84de0f034ab6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala @@ -31,84 +31,86 @@ class FoldablePropagationSuite extends PlanTest { FoldablePropagation) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int) test("Propagate from subquery") { val query = OneRowRelation() - .select(Literal(1).as('a), Literal(2).as('b)) - .subquery('T) - .select('a, 'b) + .select(Literal(1).as("a"), Literal(2).as("b")) + .subquery("T") + .select("a".attr, "b".attr) val optimized = Optimize.execute(query.analyze) val correctAnswer = OneRowRelation() - .select(Literal(1).as('a), Literal(2).as('b)) - .subquery('T) - .select(Literal(1).as('a), Literal(2).as('b)).analyze + .select(Literal(1).as("a"), Literal(2).as("b")) + .subquery("T") + .select(Literal(1).as("a"), Literal(2).as("b")).analyze comparePlans(optimized, correctAnswer) } test("Propagate to select clause") { val query = testRelation - .select('a.as('x), "str".as('y), 'b.as('z)) - .select('x, 'y, 'z) + .select("a".attr.as("x"), "str".as("y"), "b".attr.as("z")) + .select("x".attr, "y".attr, "z".attr) val optimized = Optimize.execute(query.analyze) val correctAnswer = testRelation - .select('a.as('x), "str".as('y), 'b.as('z)) - .select('x, "str".as('y), 'z).analyze + .select("a".attr.as("x"), "str".as("y"), "b".attr.as("z")) + .select("x".attr, "str".as("y"), "z".attr).analyze comparePlans(optimized, correctAnswer) } test("Propagate to where clause") { val query = testRelation - .select("str".as('y)) - .where('y === "str" && "str" === 'y) + .select("str".as("y")) + .where("y".attr === "str" && "str" === "y".attr) val optimized = Optimize.execute(query.analyze) val correctAnswer = testRelation - .select("str".as('y)) - .where("str".as('y) === "str" && "str" === "str".as('y)).analyze + .select("str".as("y")) + .where("str".as("y") === "str" && "str" === "str".as("y")).analyze comparePlans(optimized, correctAnswer) } test("Propagate to orderBy clause") { val query = testRelation - .select('a.as('x), Year(CurrentDate()).as('y), 'b) - .orderBy('x.asc, 'y.asc, 'b.desc) + .select("a".attr.as("x"), Year(CurrentDate()).as("y"), "b".attr) + .orderBy("x".attr.asc, "y".attr.asc, "b".attr.desc) val optimized = Optimize.execute(query.analyze) val correctAnswer = testRelation - .select('a.as('x), Year(CurrentDate()).as('y), 'b) - .orderBy('x.asc, SortOrder(Year(CurrentDate()), Ascending), 'b.desc).analyze + .select("a".attr.as("x"), Year(CurrentDate()).as("y"), "b".attr) + .orderBy("x".attr.asc, SortOrder(Year(CurrentDate()), Ascending), "b".attr.desc).analyze comparePlans(optimized, correctAnswer) } test("Propagate to groupBy clause") { val query = testRelation - .select('a.as('x), Year(CurrentDate()).as('y), 'b) - .groupBy('x, 'y, 'b)(sum('x), avg('y).as('AVG), count('b)) + .select("a".attr.as("x"), Year(CurrentDate()).as("y"), "b".attr) + .groupBy("x".attr, "y".attr, "b".attr)(sum("x".attr), + avg("y".attr).as("AVG"), count("b".attr)) val optimized = Optimize.execute(query.analyze) val correctAnswer = testRelation - .select('a.as('x), Year(CurrentDate()).as('y), 'b) - .groupBy('x, Year(CurrentDate()).as('y), 'b)(sum('x), avg(Year(CurrentDate())).as('AVG), - count('b)).analyze + .select("a".attr.as("x"), Year(CurrentDate()).as("y"), "b".attr) + .groupBy("x".attr, Year(CurrentDate()).as("y"), "b".attr)(sum("x".attr), + avg(Year(CurrentDate())).as("AVG"), count("b".attr)).analyze comparePlans(optimized, correctAnswer) } test("Propagate in a complex query") { val query = testRelation - .select('a.as('x), Year(CurrentDate()).as('y), 'b) - .where('x > 1 && 'y === 2016 && 'b > 1) - .groupBy('x, 'y, 'b)(sum('x), avg('y).as('AVG), count('b)) - .orderBy('x.asc, 'AVG.asc) + .select("a".attr.as("x"), Year(CurrentDate()).as("y"), "b".attr) + .where("x".attr > 1 && "y".attr === 2016 && "b".attr > 1) + .groupBy("x".attr, "y".attr, "b".attr)(sum("x".attr), + avg("y".attr).as("AVG"), count("b".attr)) + .orderBy("x".attr.asc, "AVG".attr.asc) val optimized = Optimize.execute(query.analyze) val correctAnswer = testRelation - .select('a.as('x), Year(CurrentDate()).as('y), 'b) - .where('x > 1 && Year(CurrentDate()).as('y) === 2016 && 'b > 1) - .groupBy('x, Year(CurrentDate()).as("y"), 'b)(sum('x), avg(Year(CurrentDate())).as('AVG), - count('b)) - .orderBy('x.asc, 'AVG.asc).analyze + .select("a".attr.as("x"), Year(CurrentDate()).as("y"), "b".attr) + .where("x".attr > 1 && Year(CurrentDate()).as("y") === 2016 && "b".attr > 1) + .groupBy("x".attr, Year(CurrentDate()).as("y"), "b".attr)(sum("x".attr), + avg(Year(CurrentDate())).as("AVG"), count("b".attr)) + .orderBy("x".attr.asc, "AVG".attr.asc).analyze comparePlans(optimized, correctAnswer) } @@ -116,27 +118,27 @@ class FoldablePropagationSuite extends PlanTest { test("Propagate in subqueries of Union queries") { val query = Union( Seq( - testRelation.select(Literal(1).as('x), 'a).select('x, 'x + 'a), - testRelation.select(Literal(2).as('x), 'a).select('x, 'x + 'a))) - .select('x) + testRelation.select(Literal(1).as("x"), "a".attr).select("x".attr, "x".attr + "a".attr), + testRelation.select(Literal(2).as("x"), "a".attr).select("x".attr, "x".attr + "a".attr))) + .select("x".attr) val optimized = Optimize.execute(query.analyze) val correctAnswer = Union( Seq( - testRelation.select(Literal(1).as('x), 'a) - .select(Literal(1).as('x), (Literal(1).as('x) + 'a).as("(x + a)")), - testRelation.select(Literal(2).as('x), 'a) - .select(Literal(2).as('x), (Literal(2).as('x) + 'a).as("(x + a)")))) - .select('x).analyze + testRelation.select(Literal(1).as("x"), "a".attr) + .select(Literal(1).as("x"), (Literal(1).as("x") + "a".attr).as("(x + a)")), + testRelation.select(Literal(2).as("x"), "a".attr) + .select(Literal(2).as("x"), (Literal(2).as("x") + "a".attr).as("(x + a)")))) + .select("x".attr).analyze comparePlans(optimized, correctAnswer) } test("Propagate in inner join") { - val ta = testRelation.select('a, Literal(1).as('tag)) - .union(testRelation.select('a.as('a), Literal(2).as('tag))) - .subquery('ta) - val tb = testRelation.select('a, Literal(1).as('tag)) - .union(testRelation.select('a.as('a), Literal(2).as('tag))) - .subquery('tb) + val ta = testRelation.select("a".attr, Literal(1).as("tag")) + .union(testRelation.select("a".attr.as("a"), Literal(2).as("tag"))) + .subquery("ta") + val tb = testRelation.select("a".attr, Literal(1).as("tag")) + .union(testRelation.select("a".attr.as("a"), Literal(2).as("tag"))) + .subquery("tb") val query = ta.join(tb, Inner, Some("ta.a".attr === "tb.a".attr && "ta.tag".attr === "tb.tag".attr)) val optimized = Optimize.execute(query.analyze) @@ -145,12 +147,12 @@ class FoldablePropagationSuite extends PlanTest { } test("Propagate in expand") { - val c1 = Literal(1).as('a) - val c2 = Literal(2).as('b) + val c1 = Literal(1).as("a") + val c2 = Literal(2).as("b") val a1 = c1.toAttribute.newInstance().withNullability(true) val a2 = c2.toAttribute.newInstance().withNullability(true) val expand = Expand( - Seq(Seq(Literal(null), 'b), Seq('a, Literal(null))), + Seq(Seq(Literal(null), "b".attr), Seq("a".attr, Literal(null))), Seq(a1, a2), OneRowRelation().select(c1, c2)) val query = expand.where(a1.isNotNull).select(a1, a2).analyze @@ -163,30 +165,30 @@ class FoldablePropagationSuite extends PlanTest { } test("Propagate above outer join") { - val left = LocalRelation('a.int).select('a, Literal(1).as('b)) - val right = LocalRelation('c.int).select('c, Literal(1).as('d)) + val left = LocalRelation("a".attr.int).select("a".attr, Literal(1).as("b")) + val right = LocalRelation("c".attr.int).select("c".attr, Literal(1).as("d")) val join = left.join( right, joinType = LeftOuter, - condition = Some('a === 'c && 'b === 'd)) - val query = join.select(('b + 3).as('res)).analyze + condition = Some("a".attr === "c".attr && "b".attr === "d".attr)) + val query = join.select(("b".attr + 3).as("res")).analyze val optimized = Optimize.execute(query) val correctAnswer = left.join( right, joinType = LeftOuter, - condition = Some('a === 'c && Literal(1) === Literal(1))) - .select((Literal(1) + 3).as('res)).analyze + condition = Some("a".attr === "c".attr && Literal(1) === Literal(1))) + .select((Literal(1) + 3).as("res")).analyze comparePlans(optimized, correctAnswer) } test("SPARK-32635: Replace references with foldables coming only from the node's children") { - val leftExpression = 'a.int - val left = LocalRelation(leftExpression).select('a) + val leftExpression = "a".attr.int + val left = LocalRelation(leftExpression).select("a".attr) val rightExpression = Alias(Literal(2), "a")(leftExpression.exprId) - val right = LocalRelation('b.int).select('b, rightExpression).select('b) - val join = left.join(right, joinType = LeftOuter, condition = Some('b === 'a)) + val right = LocalRelation("b".attr.int).select("b".attr, rightExpression).select("b".attr) + val join = left.join(right, joinType = LeftOuter, condition = Some("b".attr === "a".attr)) val query = join.analyze val optimized = Optimize.execute(query) @@ -195,13 +197,13 @@ class FoldablePropagationSuite extends PlanTest { test("SPARK-32951: Foldable propagation from Aggregate") { val query = testRelation - .groupBy('a)('a, sum('b).as('b), Literal(1).as('c)) - .select('a, 'b, 'c) + .groupBy("a".attr)("a".attr, sum("b".attr).as("b"), Literal(1).as("c")) + .select("a".attr, "b".attr, "c".attr) val optimized = Optimize.execute(query.analyze) val correctAnswer = testRelation - .groupBy('a)('a, sum('b).as('b), Literal(1).as('c)) - .select('a, 'b, Literal(1).as('c)).analyze + .groupBy("a".attr)("a".attr, sum("b".attr).as("b"), Literal(1).as("c")) + .select("a".attr, "b".attr, Literal(1).as("c")).analyze comparePlans(optimized, correctAnswer) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala index 79bd573f1d84..ebdf8af5a597 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala @@ -40,7 +40,7 @@ class InferFiltersFromConstraintsSuite extends PlanTest { PruneFilters) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) private def testConstraintsAfterJoin( x: LogicalPlan, @@ -56,46 +56,51 @@ class InferFiltersFromConstraintsSuite extends PlanTest { } test("filter: filter out constraints in condition") { - val originalQuery = testRelation.where('a === 1 && 'a === 'b).analyze + val originalQuery = testRelation.where("a".attr === 1 && "a".attr === "b".attr).analyze val correctAnswer = testRelation - .where(IsNotNull('a) && IsNotNull('b) && 'a === 'b && 'a === 1 && 'b === 1).analyze + .where(IsNotNull("a".attr) && IsNotNull("b".attr) && "a".attr === "b".attr && + "a".attr === 1 && "b".attr === 1) + .analyze val optimized = Optimize.execute(originalQuery) comparePlans(optimized, correctAnswer) } test("single inner join: filter out values on either side on equi-join keys") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = x.join(y, condition = Some(("x.a".attr === "y.a".attr) && ("x.a".attr === 1) && ("y.c".attr > 5))) .analyze - val left = x.where(IsNotNull('a) && "x.a".attr === 1) - val right = y.where(IsNotNull('a) && IsNotNull('c) && "y.c".attr > 5 && "y.a".attr === 1) + val left = x.where(IsNotNull("a".attr) && "x.a".attr === 1) + val right = + y.where(IsNotNull("a".attr) && IsNotNull("c".attr) && "y.c".attr > 5 && "y.a".attr === 1) val correctAnswer = left.join(right, condition = Some("x.a".attr === "y.a".attr)).analyze val optimized = Optimize.execute(originalQuery) comparePlans(optimized, correctAnswer) } test("single inner join: filter out nulls on either side on non equal keys") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = x.join(y, condition = Some(("x.a".attr =!= "y.a".attr) && ("x.b".attr === 1) && ("y.c".attr > 5))) .analyze - val left = x.where(IsNotNull('a) && IsNotNull('b) && "x.b".attr === 1) - val right = y.where(IsNotNull('a) && IsNotNull('c) && "y.c".attr > 5) + val left = x.where(IsNotNull("a".attr) && IsNotNull("b".attr) && "x.b".attr === 1) + val right = y.where(IsNotNull("a".attr) && IsNotNull("c".attr) && "y.c".attr > 5) val correctAnswer = left.join(right, condition = Some("x.a".attr =!= "y.a".attr)).analyze val optimized = Optimize.execute(originalQuery) comparePlans(optimized, correctAnswer) } test("single inner join with pre-existing filters: filter out values on either side") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) - val originalQuery = x.where('b > 5).join(y.where('a === 10), + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") + val originalQuery = x.where("b".attr > 5).join(y.where("a".attr === 10), condition = Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)).analyze - val left = x.where(IsNotNull('a) && 'a === 10 && IsNotNull('b) && 'b > 5) - val right = y.where(IsNotNull('a) && IsNotNull('b) && 'a === 10 && 'b > 5) + val left = + x.where(IsNotNull("a".attr) && "a".attr === 10 && IsNotNull("b".attr) && "b".attr > 5) + val right = + y.where(IsNotNull("a".attr) && IsNotNull("b".attr) && "a".attr === 10 && "b".attr > 5) val correctAnswer = left.join(right, condition = Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)).analyze val optimized = Optimize.execute(originalQuery) @@ -103,8 +108,8 @@ class InferFiltersFromConstraintsSuite extends PlanTest { } test("single outer join: no null filters are generated") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = x.join(y, FullOuter, condition = Some("x.a".attr === "y.a".attr)).analyze val optimized = Optimize.execute(originalQuery) @@ -112,47 +117,52 @@ class InferFiltersFromConstraintsSuite extends PlanTest { } test("multiple inner joins: filter out values on all sides on equi-join keys") { - val t1 = testRelation.subquery('t1) - val t2 = testRelation.subquery('t2) - val t3 = testRelation.subquery('t3) - val t4 = testRelation.subquery('t4) + val t1 = testRelation.subquery("t1") + val t2 = testRelation.subquery("t2") + val t3 = testRelation.subquery("t3") + val t4 = testRelation.subquery("t4") - val originalQuery = t1.where('b > 5) + val originalQuery = t1.where("b".attr > 5) .join(t2, condition = Some("t1.b".attr === "t2.b".attr)) .join(t3, condition = Some("t2.b".attr === "t3.b".attr)) .join(t4, condition = Some("t3.b".attr === "t4.b".attr)).analyze - val correctAnswer = t1.where(IsNotNull('b) && 'b > 5) - .join(t2.where(IsNotNull('b) && 'b > 5), condition = Some("t1.b".attr === "t2.b".attr)) - .join(t3.where(IsNotNull('b) && 'b > 5), condition = Some("t2.b".attr === "t3.b".attr)) - .join(t4.where(IsNotNull('b) && 'b > 5), condition = Some("t3.b".attr === "t4.b".attr)) + val correctAnswer = t1.where(IsNotNull("b".attr) && "b".attr > 5) + .join(t2.where(IsNotNull("b".attr) && "b".attr > 5), + condition = Some("t1.b".attr === "t2.b".attr)) + .join(t3.where(IsNotNull("b".attr) && "b".attr > 5), + condition = Some("t2.b".attr === "t3.b".attr)) + .join(t4.where(IsNotNull("b".attr) && "b".attr > 5), + condition = Some("t3.b".attr === "t4.b".attr)) .analyze val optimized = Optimize.execute(originalQuery) comparePlans(optimized, correctAnswer) } test("inner join with filter: filter out values on all sides on equi-join keys") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val originalQuery = x.join(y, Inner, Some("x.a".attr === "y.a".attr)).where("x.a".attr > 5).analyze - val correctAnswer = x.where(IsNotNull('a) && 'a.attr > 5) - .join(y.where(IsNotNull('a) && 'a.attr > 5), Inner, Some("x.a".attr === "y.a".attr)).analyze + val correctAnswer = x.where(IsNotNull("a".attr) && "a".attr > 5) + .join(y.where(IsNotNull("a".attr) && "a".attr > 5), Inner, + Some("x.a".attr === "y.a".attr)).analyze val optimized = Optimize.execute(originalQuery) comparePlans(optimized, correctAnswer) } test("inner join with alias: alias contains multiple attributes") { - val t1 = testRelation.subquery('t1) - val t2 = testRelation.subquery('t2) + val t1 = testRelation.subquery("t1") + val t2 = testRelation.subquery("t2") - val originalQuery = t1.select('a, Coalesce(Seq('a, 'b)).as('int_col)).as("t") + val originalQuery = t1.select("a".attr, Coalesce(Seq("a".attr, "b".attr)).as("int_col")).as("t") .join(t2, Inner, Some("t.a".attr === "t2.a".attr && "t.int_col".attr === "t2.a".attr)) .analyze val correctAnswer = t1 - .where(IsNotNull('a) && IsNotNull(Coalesce(Seq('a, 'b))) && 'a === Coalesce(Seq('a, 'b))) - .select('a, Coalesce(Seq('a, 'b)).as('int_col)).as("t") - .join(t2.where(IsNotNull('a)), Inner, + .where(IsNotNull("a".attr) && IsNotNull(Coalesce(Seq("a".attr, "b".attr))) && + "a".attr === Coalesce(Seq("a".attr, "b".attr))) + .select("a".attr, Coalesce(Seq("a".attr, "b".attr)).as("int_col")).as("t") + .join(t2.where(IsNotNull("a".attr)), Inner, Some("t.a".attr === "t2.a".attr && "t.int_col".attr === "t2.a".attr)) .analyze val optimized = Optimize.execute(originalQuery) @@ -160,16 +170,16 @@ class InferFiltersFromConstraintsSuite extends PlanTest { } test("inner join with alias: alias contains single attributes") { - val t1 = testRelation.subquery('t1) - val t2 = testRelation.subquery('t2) + val t1 = testRelation.subquery("t1") + val t2 = testRelation.subquery("t2") - val originalQuery = t1.select('a, 'b.as('d)).as("t") + val originalQuery = t1.select("a".attr, "b".attr.as("d")).as("t") .join(t2, Inner, Some("t.a".attr === "t2.a".attr && "t.d".attr === "t2.a".attr)) .analyze val correctAnswer = t1 - .where(IsNotNull('a) && IsNotNull('b) &&'a === 'b) - .select('a, 'b.as('d)).as("t") - .join(t2.where(IsNotNull('a)), Inner, + .where(IsNotNull("a".attr) && IsNotNull("b".attr) &&"a".attr === "b".attr) + .select("a".attr, "b".attr.as("d")).as("t") + .join(t2.where(IsNotNull("a".attr)), Inner, Some("t.a".attr === "t2.a".attr && "t.d".attr === "t2.a".attr)) .analyze val optimized = Optimize.execute(originalQuery) @@ -177,29 +187,33 @@ class InferFiltersFromConstraintsSuite extends PlanTest { } test("generate correct filters for alias that don't produce recursive constraints") { - val t1 = testRelation.subquery('t1) + val t1 = testRelation.subquery("t1") - val originalQuery = t1.select('a.as('x), 'b.as('y)).where('x === 1 && 'x === 'y).analyze - val correctAnswer = - t1.where('a === 1 && 'b === 1 && 'a === 'b && IsNotNull('a) && IsNotNull('b)) - .select('a.as('x), 'b.as('y)).analyze + val originalQuery = t1 + .select("a".attr.as("x"), "b".attr.as("y")) + .where("x".attr === 1 && "x".attr === "y".attr).analyze + val correctAnswer = t1 + .where("a".attr === 1 && "b".attr === 1 && "a".attr === "b".attr && + IsNotNull("a".attr) && IsNotNull("b".attr)) + .select("a".attr.as("x"), "b".attr.as("y")).analyze val optimized = Optimize.execute(originalQuery) comparePlans(optimized, correctAnswer) } test("No inferred filter when constraint propagation is disabled") { withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "false") { - val originalQuery = testRelation.where('a === 1 && 'a === 'b).analyze + val originalQuery = testRelation.where("a".attr === 1 && "a".attr === "b".attr).analyze val optimized = Optimize.execute(originalQuery) comparePlans(optimized, originalQuery) } } test("constraints should be inferred from aliased literals") { - val originalLeft = testRelation.subquery('left).as("left") - val optimizedLeft = testRelation.subquery('left).where(IsNotNull('a) && 'a <=> 2).as("left") + val originalLeft = testRelation.subquery("left").as("left") + val optimizedLeft = + testRelation.subquery("left").where(IsNotNull("a".attr) && "a".attr <=> 2).as("left") - val right = Project(Seq(Literal(2).as("two")), testRelation.subquery('right)).as("right") + val right = Project(Seq(Literal(2).as("two")), testRelation.subquery("right")).as("right") val condition = Some("left.a".attr === "right.two".attr) val original = originalLeft.join(right, Inner, condition) @@ -209,70 +223,72 @@ class InferFiltersFromConstraintsSuite extends PlanTest { } test("SPARK-23405: left-semi equal-join should filter out null join keys on both sides") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) - testConstraintsAfterJoin(x, y, x.where(IsNotNull('a)), y.where(IsNotNull('a)), LeftSemi) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") + testConstraintsAfterJoin(x, y, x.where(IsNotNull("a".attr)), + y.where(IsNotNull("a".attr)), LeftSemi) } test("SPARK-21479: Outer join after-join filters push down to null-supplying side") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val condition = Some("x.a".attr === "y.a".attr) val originalQuery = x.join(y, LeftOuter, condition).where("x.a".attr === 2).analyze - val left = x.where(IsNotNull('a) && 'a === 2) - val right = y.where(IsNotNull('a) && 'a === 2) + val left = x.where(IsNotNull("a".attr) && "a".attr === 2) + val right = y.where(IsNotNull("a".attr) && "a".attr === 2) val correctAnswer = left.join(right, LeftOuter, condition).analyze val optimized = Optimize.execute(originalQuery) comparePlans(optimized, correctAnswer) } test("SPARK-21479: Outer join pre-existing filters push down to null-supplying side") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val condition = Some("x.a".attr === "y.a".attr) val originalQuery = x.join(y.where("y.a".attr > 5), RightOuter, condition).analyze - val left = x.where(IsNotNull('a) && 'a > 5) - val right = y.where(IsNotNull('a) && 'a > 5) + val left = x.where(IsNotNull("a".attr) && "a".attr > 5) + val right = y.where(IsNotNull("a".attr) && "a".attr > 5) val correctAnswer = left.join(right, RightOuter, condition).analyze val optimized = Optimize.execute(originalQuery) comparePlans(optimized, correctAnswer) } test("SPARK-21479: Outer join no filter push down to preserved side") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") testConstraintsAfterJoin( x, y.where("a".attr === 1), - x, y.where(IsNotNull('a) && 'a === 1), + x, y.where(IsNotNull("a".attr) && "a".attr === 1), LeftOuter) } test("SPARK-23564: left anti join should filter out null join keys on right side") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) - testConstraintsAfterJoin(x, y, x, y.where(IsNotNull('a)), LeftAnti) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") + testConstraintsAfterJoin(x, y, x, y.where(IsNotNull("a".attr)), LeftAnti) } test("SPARK-23564: left outer join should filter out null join keys on right side") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) - testConstraintsAfterJoin(x, y, x, y.where(IsNotNull('a)), LeftOuter) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") + testConstraintsAfterJoin(x, y, x, y.where(IsNotNull("a".attr)), LeftOuter) } test("SPARK-23564: right outer join should filter out null join keys on left side") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) - testConstraintsAfterJoin(x, y, x.where(IsNotNull('a)), y, RightOuter) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") + testConstraintsAfterJoin(x, y, x.where(IsNotNull("a".attr)), y, RightOuter) } test("Constraints should be inferred from cast equality constraint(filter higher data type)") { - val testRelation1 = LocalRelation('a.int) - val testRelation2 = LocalRelation('b.long) - val originalLeft = testRelation1.subquery('left) - val originalRight = testRelation2.where('b === 1L).subquery('right) + val testRelation1 = LocalRelation("a".attr.int) + val testRelation2 = LocalRelation("b".attr.long) + val originalLeft = testRelation1.subquery("left") + val originalRight = testRelation2.where("b".attr === 1L).subquery("right") - val left = testRelation1.where(IsNotNull('a) && 'a.cast(LongType) === 1L).subquery('left) - val right = testRelation2.where(IsNotNull('b) && 'b === 1L).subquery('right) + val left = + testRelation1.where(IsNotNull("a".attr) && "a".attr.cast(LongType) === 1L).subquery("left") + val right = testRelation2.where(IsNotNull("b".attr) && "b".attr === 1L).subquery("right") Seq(Some("left.a".attr.cast(LongType) === "right.b".attr), Some("right.b".attr === "left.a".attr.cast(LongType))).foreach { condition => @@ -284,7 +300,7 @@ class InferFiltersFromConstraintsSuite extends PlanTest { testConstraintsAfterJoin( originalLeft, originalRight, - testRelation1.where(IsNotNull('a)).subquery('left), + testRelation1.where(IsNotNull("a".attr)).subquery("left"), right, Inner, condition) @@ -292,13 +308,13 @@ class InferFiltersFromConstraintsSuite extends PlanTest { } test("Constraints shouldn't be inferred from cast equality constraint(filter lower data type)") { - val testRelation1 = LocalRelation('a.int) - val testRelation2 = LocalRelation('b.long) - val originalLeft = testRelation1.where('a === 1).subquery('left) - val originalRight = testRelation2.subquery('right) + val testRelation1 = LocalRelation("a".attr.int) + val testRelation2 = LocalRelation("b".attr.long) + val originalLeft = testRelation1.where("a".attr === 1).subquery("left") + val originalRight = testRelation2.subquery("right") - val left = testRelation1.where(IsNotNull('a) && 'a === 1).subquery('left) - val right = testRelation2.where(IsNotNull('b)).subquery('right) + val left = testRelation1.where(IsNotNull("a".attr) && "a".attr === 1).subquery("left") + val right = testRelation2.where(IsNotNull("b".attr)).subquery("right") Seq(Some("left.a".attr.cast(LongType) === "right.b".attr), Some("right.b".attr === "left.a".attr.cast(LongType))).foreach { condition => @@ -311,7 +327,9 @@ class InferFiltersFromConstraintsSuite extends PlanTest { originalLeft, originalRight, left, - testRelation2.where(IsNotNull('b) && 'b.attr.cast(IntegerType) === 1).subquery('right), + testRelation2 + .where(IsNotNull("b".attr) && "b".attr.cast(IntegerType) === 1) + .subquery("right"), Inner, condition) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromGenerateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromGenerateSuite.scala index 93a1d414ed40..0cfc64d15107 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromGenerateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromGenerateSuite.scala @@ -31,17 +31,17 @@ class InferFiltersFromGenerateSuite extends PlanTest { val batches = Batch("Infer Filters", Once, InferFiltersFromGenerate) :: Nil } - val testRelation = LocalRelation('a.array(StructType(Seq( + val testRelation = LocalRelation("a".attr.array(StructType(Seq( StructField("x", IntegerType), StructField("y", IntegerType) - ))), 'c1.string, 'c2.string) + ))), "c1".attr.string, "c2".attr.string) Seq(Explode(_), PosExplode(_), Inline(_)).foreach { f => - val generator = f('a) + val generator = f("a".attr) test("Infer filters from " + generator) { val originalQuery = testRelation.generate(generator).analyze val correctAnswer = testRelation - .where(IsNotNull('a) && Size('a) > 0) + .where(IsNotNull("a".attr) && Size("a".attr) > 0) .generate(generator) .analyze val optimized = Optimize.execute(originalQuery) @@ -50,7 +50,7 @@ class InferFiltersFromGenerateSuite extends PlanTest { test("Don't infer duplicate filters from " + generator) { val originalQuery = testRelation - .where(IsNotNull('a) && Size('a) > 0) + .where(IsNotNull("a".attr) && Size("a".attr) > 0) .generate(generator) .analyze val optimized = Optimize.execute(originalQuery) @@ -89,13 +89,13 @@ class InferFiltersFromGenerateSuite extends PlanTest { } Seq(Explode(_), PosExplode(_)).foreach { f => - val createArrayExplode = f(CreateArray(Seq('c1))) + val createArrayExplode = f(CreateArray(Seq("c1".attr))) test("SPARK-33544: Don't infer filters from CreateArray " + createArrayExplode) { val originalQuery = testRelation.generate(createArrayExplode).analyze val optimized = OptimizeInferAndConstantFold.execute(originalQuery) comparePlans(optimized, originalQuery) } - val createMapExplode = f(CreateMap(Seq('c1, 'c2))) + val createMapExplode = f(CreateMap(Seq("c1".attr, "c2".attr))) test("SPARK-33544: Don't infer filters from CreateMap " + createMapExplode) { val originalQuery = testRelation.generate(createMapExplode).analyze val optimized = OptimizeInferAndConstantFold.execute(originalQuery) @@ -104,7 +104,7 @@ class InferFiltersFromGenerateSuite extends PlanTest { } Seq(Inline(_)).foreach { f => - val createArrayStructExplode = f(CreateArray(Seq(CreateStruct(Seq('c1))))) + val createArrayStructExplode = f(CreateArray(Seq(CreateStruct(Seq("c1".attr))))) test("SPARK-33544: Don't infer filters from CreateArray " + createArrayStructExplode) { val originalQuery = testRelation.generate(createArrayStructExplode).analyze val optimized = OptimizeInferAndConstantFold.execute(originalQuery) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala index 3d81c567eff1..8a6e53e06f78 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala @@ -44,13 +44,13 @@ class JoinOptimizationSuite extends PlanTest { } - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) - val testRelation1 = LocalRelation('d.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) + val testRelation1 = LocalRelation("d".attr.int) test("extract filters and joins") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) - val z = testRelation.subquery('z) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") + val z = testRelation.subquery("z") def testExtract(plan: LogicalPlan, expected: Option[(Seq[LogicalPlan], Seq[Expression])]): Unit = { @@ -96,9 +96,9 @@ class JoinOptimizationSuite extends PlanTest { } test("reorder inner joins") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) - val z = testRelation.subquery('z) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") + val z = testRelation.subquery("z") val queryAnswers = Seq( ( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinSelectionHelperSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinSelectionHelperSuite.scala index 3513cfa14808..ad343d95d9b1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinSelectionHelperSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinSelectionHelperSuite.scala @@ -27,13 +27,13 @@ import org.apache.spark.sql.internal.SQLConf class JoinSelectionHelperSuite extends PlanTest with JoinSelectionHelper { private val left = StatsTestPlan( - outputList = Seq('a.int, 'b.int, 'c.int), + outputList = Seq("a".attr.int, "b".attr.int, "c".attr.int), rowCount = 20000000, size = Some(20000000), attributeStats = AttributeMap(Seq())) private val right = StatsTestPlan( - outputList = Seq('d.int), + outputList = Seq("d".attr.int), rowCount = 1000, size = Some(1000), attributeStats = AttributeMap(Seq())) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala index 1672c6d91660..1ebd3b044c4a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala @@ -42,41 +42,41 @@ class LeftSemiPushdownSuite extends PlanTest { CollapseProject) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) - val testRelation1 = LocalRelation('d.int) - val testRelation2 = LocalRelation('e.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) + val testRelation1 = LocalRelation("d".attr.int) + val testRelation2 = LocalRelation("e".attr.int) test("Project: LeftSemiAnti join pushdown") { val originalQuery = testRelation .select(star()) - .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) + .join(testRelation1, joinType = LeftSemi, condition = Some("b".attr === "d".attr)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) - .select('a, 'b, 'c) + .join(testRelation1, joinType = LeftSemi, condition = Some("b".attr === "d".attr)) + .select("a".attr, "b".attr, "c".attr) .analyze comparePlans(optimized, correctAnswer) } test("Project: LeftSemiAnti join no pushdown because of non-deterministic proj exprs") { val originalQuery = testRelation - .select(Rand(1), 'b, 'c) - .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) + .select(Rand(1), "b".attr, "c".attr) + .join(testRelation1, joinType = LeftSemi, condition = Some("b".attr === "d".attr)) val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, originalQuery.analyze) } test("Project: LeftSemiAnti join non correlated scalar subq") { - val subq = ScalarSubquery(testRelation.groupBy('b)(sum('c).as("sum")).analyze) + val subq = ScalarSubquery(testRelation.groupBy("b".attr)(sum("c".attr).as("sum")).analyze) val originalQuery = testRelation .select(subq.as("sum")) - .join(testRelation1, joinType = LeftSemi, condition = Some('sum === 'd)) + .join(testRelation1, joinType = LeftSemi, condition = Some("sum".attr === "d".attr)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .join(testRelation1, joinType = LeftSemi, condition = Some(subq === 'd)) + .join(testRelation1, joinType = LeftSemi, condition = Some(subq === "d".attr)) .select(subq.as("sum")) .analyze @@ -84,12 +84,13 @@ class LeftSemiPushdownSuite extends PlanTest { } test("Project: LeftSemiAnti join no pushdown - correlated scalar subq in projection list") { - val testRelation2 = LocalRelation('e.int, 'f.int) - val subqPlan = testRelation2.groupBy('e)(sum('f).as("sum")).where('e === 'a) + val testRelation2 = LocalRelation("e".attr.int, "f".attr.int) + val subqPlan = + testRelation2.groupBy("e".attr)(sum("f".attr).as("sum")).where("e".attr === "a".attr) val subqExpr = ScalarSubquery(subqPlan) val originalQuery = testRelation .select(subqExpr.as("sum")) - .join(testRelation1, joinType = LeftSemi, condition = Some('sum === 'd)) + .join(testRelation1, joinType = LeftSemi, condition = Some("sum".attr === "d".attr)) val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, originalQuery.analyze) @@ -97,13 +98,13 @@ class LeftSemiPushdownSuite extends PlanTest { test("Aggregate: LeftSemiAnti join pushdown") { val originalQuery = testRelation - .groupBy('b)('b, sum('c)) - .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) + .groupBy("b".attr)("b".attr, sum("c".attr)) + .join(testRelation1, joinType = LeftSemi, condition = Some("b".attr === "d".attr)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) - .groupBy('b)('b, sum('c)) + .join(testRelation1, joinType = LeftSemi, condition = Some("b".attr === "d".attr)) + .groupBy("b".attr)("b".attr, sum("c".attr)) .analyze comparePlans(optimized, correctAnswer) @@ -111,8 +112,8 @@ class LeftSemiPushdownSuite extends PlanTest { test("Aggregate: LeftSemiAnti join no pushdown due to non-deterministic aggr expressions") { val originalQuery = testRelation - .groupBy('b)('b, Rand(10).as('c)) - .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) + .groupBy("b".attr)("b".attr, Rand(10).as("c")) + .join(testRelation1, joinType = LeftSemi, condition = Some("b".attr === "d".attr)) val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, originalQuery.analyze) @@ -120,14 +121,15 @@ class LeftSemiPushdownSuite extends PlanTest { test("Aggregate: LeftSemi join partial pushdown") { val originalQuery = testRelation - .groupBy('b)('b, sum('c).as('sum)) - .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd && 'sum === 10)) + .groupBy("b".attr)("b".attr, sum("c".attr).as("sum")) + .join(testRelation1, joinType = LeftSemi, + condition = Some("b".attr === "d".attr && "sum".attr === 10)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) - .groupBy('b)('b, sum('c).as('sum)) - .where('sum === 10) + .join(testRelation1, joinType = LeftSemi, condition = Some("b".attr === "d".attr)) + .groupBy("b".attr)("b".attr, sum("c".attr).as("sum")) + .where("sum".attr === 10) .analyze comparePlans(optimized, correctAnswer) @@ -135,8 +137,9 @@ class LeftSemiPushdownSuite extends PlanTest { test("Aggregate: LeftAnti join no pushdown") { val originalQuery = testRelation - .groupBy('b)('b, sum('c).as('sum)) - .join(testRelation1, joinType = LeftAnti, condition = Some('b === 'd && 'sum === 10)) + .groupBy("b".attr)("b".attr, sum("c".attr).as("sum")) + .join(testRelation1, joinType = LeftAnti, + condition = Some("b".attr === "d".attr && "sum".attr === 10)) val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, originalQuery.analyze) @@ -144,42 +147,46 @@ class LeftSemiPushdownSuite extends PlanTest { test("LeftSemiAnti join over aggregate - no pushdown") { val originalQuery = testRelation - .groupBy('b)('b, sum('c).as('sum)) - .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd && 'sum === 'd)) + .groupBy("b".attr)("b".attr, sum("c".attr).as("sum")) + .join(testRelation1, joinType = LeftSemi, + condition = Some("b".attr === "d".attr && "sum".attr === "d".attr)) val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, originalQuery.analyze) } test("Aggregate: LeftSemiAnti join non-correlated scalar subq aggr exprs") { - val subq = ScalarSubquery(testRelation.groupBy('b)(sum('c).as("sum")).analyze) + val subq = ScalarSubquery(testRelation.groupBy("b".attr)(sum("c".attr).as("sum")).analyze) val originalQuery = testRelation - .groupBy('a) ('a, subq.as("sum")) - .join(testRelation1, joinType = LeftSemi, condition = Some('sum === 'd && 'a === 'd)) + .groupBy("a".attr) ("a".attr, subq.as("sum")) + .join(testRelation1, joinType = LeftSemi, + condition = Some("sum".attr === "d".attr && "a".attr === "d".attr)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .join(testRelation1, joinType = LeftSemi, condition = Some(subq === 'd && 'a === 'd)) - .groupBy('a) ('a, subq.as("sum")) + .join(testRelation1, joinType = LeftSemi, + condition = Some(subq === "d".attr && "a".attr === "d".attr)) + .groupBy("a".attr) ("a".attr, subq.as("sum")) .analyze comparePlans(optimized, correctAnswer) } test("LeftSemiAnti join over Window") { - val winExpr = windowExpr(count('b), windowSpec('a :: Nil, 'b.asc :: Nil, UnspecifiedFrame)) + val winExpr = windowExpr(count("b".attr), + windowSpec("a".attr :: Nil, "b".attr.asc :: Nil, UnspecifiedFrame)) val originalQuery = testRelation - .select('a, 'b, 'c, winExpr.as('window)) - .join(testRelation1, joinType = LeftSemi, condition = Some('a === 'd)) + .select("a".attr, "b".attr, "c".attr, winExpr.as("window")) + .join(testRelation1, joinType = LeftSemi, condition = Some("a".attr === "d".attr)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .join(testRelation1, joinType = LeftSemi, condition = Some('a === 'd)) - .select('a, 'b, 'c) - .window(winExpr.as('window) :: Nil, 'a :: Nil, 'b.asc :: Nil) - .select('a, 'b, 'c, 'window).analyze + .join(testRelation1, joinType = LeftSemi, condition = Some("a".attr === "d".attr)) + .select("a".attr, "b".attr, "c".attr) + .window(winExpr.as("window") :: Nil, "a".attr :: Nil, "b".attr.asc :: Nil) + .select("a".attr, "b".attr, "c".attr, "window".attr).analyze comparePlans(optimized, correctAnswer) } @@ -187,20 +194,22 @@ class LeftSemiPushdownSuite extends PlanTest { test("Window: LeftSemi partial pushdown") { // Attributes from join condition which does not refer to the window partition spec // are kept up in the plan as a Filter operator above Window. - val winExpr = windowExpr(count('b), windowSpec('a :: Nil, 'b.asc :: Nil, UnspecifiedFrame)) + val winExpr = + windowExpr(count("b".attr), windowSpec("a".attr :: Nil, "b".attr.asc :: Nil, UnspecifiedFrame)) val originalQuery = testRelation - .select('a, 'b, 'c, winExpr.as('window)) - .join(testRelation1, joinType = LeftSemi, condition = Some('a === 'd && 'b > 5)) + .select("a".attr, "b".attr, "c".attr, winExpr.as("window")) + .join(testRelation1, joinType = LeftSemi, + condition = Some("a".attr === "d".attr && "b".attr > 5)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .join(testRelation1, joinType = LeftSemi, condition = Some('a === 'd)) - .select('a, 'b, 'c) - .window(winExpr.as('window) :: Nil, 'a :: Nil, 'b.asc :: Nil) - .where('b > 5) - .select('a, 'b, 'c, 'window).analyze + .join(testRelation1, joinType = LeftSemi, condition = Some("a".attr === "d".attr)) + .select("a".attr, "b".attr, "c".attr) + .window(winExpr.as("window") :: Nil, "a".attr :: Nil, "b".attr.asc :: Nil) + .where("b".attr > 5) + .select("a".attr, "b".attr, "c".attr, "window".attr).analyze comparePlans(optimized, correctAnswer) } @@ -208,43 +217,48 @@ class LeftSemiPushdownSuite extends PlanTest { test("Window: LeftAnti no pushdown") { // Attributes from join condition which does not refer to the window partition spec // are kept up in the plan as a Filter operator above Window. - val winExpr = windowExpr(count('b), windowSpec('a :: Nil, 'b.asc :: Nil, UnspecifiedFrame)) + val winExpr = + windowExpr(count("b".attr), windowSpec("a".attr :: Nil, "b".attr.asc :: Nil, UnspecifiedFrame)) val originalQuery = testRelation - .select('a, 'b, 'c, winExpr.as('window)) - .join(testRelation1, joinType = LeftAnti, condition = Some('a === 'd && 'b > 5)) + .select("a".attr, "b".attr, "c".attr, winExpr.as("window")) + .join(testRelation1, joinType = LeftAnti, + condition = Some("a".attr === "d".attr && "b".attr > 5)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select('a, 'b, 'c) - .window(winExpr.as('window) :: Nil, 'a :: Nil, 'b.asc :: Nil) - .join(testRelation1, joinType = LeftAnti, condition = Some('a === 'd && 'b > 5)) - .select('a, 'b, 'c, 'window).analyze + .select("a".attr, "b".attr, "c".attr) + .window(winExpr.as("window") :: Nil, "a".attr :: Nil, "b".attr.asc :: Nil) + .join(testRelation1, joinType = LeftAnti, + condition = Some("a".attr === "d".attr && "b".attr > 5)) + .select("a".attr, "b".attr, "c".attr, "window".attr).analyze comparePlans(optimized, correctAnswer) } test("Union: LeftSemiAnti join pushdown") { - val testRelation2 = LocalRelation('x.int, 'y.int, 'z.int) + val testRelation2 = LocalRelation("x".attr.int, "y".attr.int, "z".attr.int) val originalQuery = Union(Seq(testRelation, testRelation2)) - .join(testRelation1, joinType = LeftSemi, condition = Some('a === 'd)) + .join(testRelation1, joinType = LeftSemi, condition = Some("a".attr === "d".attr)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = Union(Seq( - testRelation.join(testRelation1, joinType = LeftSemi, condition = Some('a === 'd)), - testRelation2.join(testRelation1, joinType = LeftSemi, condition = Some('x === 'd)))) + testRelation.join(testRelation1, joinType = LeftSemi, + condition = Some("a".attr === "d".attr)), + testRelation2.join(testRelation1, joinType = LeftSemi, + condition = Some("x".attr === "d".attr)))) .analyze comparePlans(optimized, correctAnswer) } test("Union: LeftSemiAnti join no pushdown in self join scenario") { - val testRelation2 = LocalRelation('x.int, 'y.int, 'z.int) + val testRelation2 = LocalRelation("x".attr.int, "y".attr.int, "z".attr.int) val originalQuery = Union(Seq(testRelation, testRelation2)) - .join(testRelation2, joinType = LeftSemi, condition = Some('a === 'x)) + .join(testRelation2, joinType = LeftSemi, condition = Some("a".attr === "x".attr)) val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, originalQuery.analyze) @@ -254,12 +268,12 @@ class LeftSemiPushdownSuite extends PlanTest { val originalQuery = testRelation .select(star()) .repartition(1) - .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) + .join(testRelation1, joinType = LeftSemi, condition = Some("b".attr === "d".attr)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) - .select('a, 'b, 'c) + .join(testRelation1, joinType = LeftSemi, condition = Some("b".attr === "d".attr)) + .select("a".attr, "b".attr, "c".attr) .repartition(1) .analyze comparePlans(optimized, correctAnswer) @@ -274,64 +288,73 @@ class LeftSemiPushdownSuite extends PlanTest { val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .join(testRelation1, joinType = LeftSemi, condition = None) - .select('a, 'b, 'c) + .select("a".attr, "b".attr, "c".attr) .repartition(1) .analyze comparePlans(optimized, correctAnswer) } test("Unary: LeftSemi join pushdown - partial pushdown") { - val testRelationWithArrayType = LocalRelation('a.int, 'b.int, 'c_arr.array(IntegerType)) + val testRelationWithArrayType = + LocalRelation("a".attr.int, "b".attr.int, "c_arr".attr.array(IntegerType)) val originalQuery = testRelationWithArrayType - .generate(Explode('c_arr), alias = Some("arr"), outputNames = Seq("out_col")) - .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd && 'b === 'out_col)) + .generate(Explode("c_arr".attr), alias = Some("arr"), outputNames = Seq("out_col")) + .join(testRelation1, joinType = LeftSemi, + condition = Some("b".attr === "d".attr && "b".attr === "out_col".attr)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelationWithArrayType - .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) - .generate(Explode('c_arr), alias = Some("arr"), outputNames = Seq("out_col")) - .where('b === 'out_col) + .join(testRelation1, joinType = LeftSemi, condition = Some("b".attr === "d".attr)) + .generate(Explode("c_arr".attr), alias = Some("arr"), outputNames = Seq("out_col")) + .where("b".attr === "out_col".attr) .analyze comparePlans(optimized, correctAnswer) } test("Unary: LeftAnti join pushdown - no pushdown") { - val testRelationWithArrayType = LocalRelation('a.int, 'b.int, 'c_arr.array(IntegerType)) + val testRelationWithArrayType = + LocalRelation("a".attr.int, "b".attr.int, "c_arr".attr.array(IntegerType)) val originalQuery = testRelationWithArrayType - .generate(Explode('c_arr), alias = Some("arr"), outputNames = Seq("out_col")) - .join(testRelation1, joinType = LeftAnti, condition = Some('b === 'd && 'b === 'out_col)) + .generate(Explode("c_arr".attr), alias = Some("arr"), outputNames = Seq("out_col")) + .join(testRelation1, joinType = LeftAnti, + condition = Some("b".attr === "d".attr && "b".attr === "out_col".attr)) val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, originalQuery.analyze) } test("Unary: LeftSemiAnti join pushdown - no pushdown") { - val testRelationWithArrayType = LocalRelation('a.int, 'b.int, 'c_arr.array(IntegerType)) + val testRelationWithArrayType = + LocalRelation("a".attr.int, "b".attr.int, "c_arr".attr.array(IntegerType)) val originalQuery = testRelationWithArrayType - .generate(Explode('c_arr), alias = Some("arr"), outputNames = Seq("out_col")) - .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd && 'd === 'out_col)) + .generate(Explode("c_arr".attr), alias = Some("arr"), outputNames = Seq("out_col")) + .join(testRelation1, joinType = LeftSemi, + condition = Some("b".attr === "d".attr && "d".attr === "out_col".attr)) val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, originalQuery.analyze) } test("Unary: LeftSemi join push down through Expand") { - val expand = Expand(Seq(Seq('a, 'b, "null"), Seq('a, "null", 'c)), - Seq('a, 'b, 'c), testRelation) + val expand = Expand(Seq(Seq("a".attr, "b".attr, "null"), Seq("a".attr, "null", "c".attr)), + Seq("a".attr, "b".attr, "c".attr), testRelation) val originalQuery = expand - .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd && 'b === 1)) + .join(testRelation1, joinType = LeftSemi, + condition = Some("b".attr === "d".attr && "b".attr === 1)) val optimized = Optimize.execute(originalQuery.analyze) - val correctAnswer = Expand(Seq(Seq('a, 'b, "null"), Seq('a, "null", 'c)), - Seq('a, 'b, 'c), testRelation - .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd && 'b === 1))) + val correctAnswer = + Expand(Seq(Seq("a".attr, "b".attr, "null"), Seq("a".attr, "null", "c".attr)), + Seq("a".attr, "b".attr, "c".attr), testRelation + .join(testRelation1, joinType = LeftSemi, + condition = Some("b".attr === "d".attr && "b".attr === 1))) .analyze comparePlans(optimized, correctAnswer) } - Seq(Some('d === 'e), None).foreach { case innerJoinCond => + Seq(Some("d".attr === "e".attr), None).foreach { case innerJoinCond => Seq(LeftSemi, LeftAnti).foreach { case outerJT => Seq(Inner, LeftOuter, Cross, RightOuter).foreach { case innerJT => test(s"$outerJT pushdown empty join cond join type $innerJT join cond $innerJoinCond") { @@ -352,17 +375,19 @@ class LeftSemiPushdownSuite extends PlanTest { } } - Seq(Some('d === 'e), None).foreach { case innerJoinCond => + Seq(Some("d".attr === "e".attr), None).foreach { case innerJoinCond => Seq(LeftSemi, LeftAnti).foreach { case outerJT => Seq(Inner, LeftOuter, Cross).foreach { case innerJT => test(s"$outerJT pushdown to left of join type: $innerJT join condition $innerJoinCond") { val joinedRelation = testRelation1.join(testRelation2, joinType = innerJT, innerJoinCond) val originalQuery = - joinedRelation.join(testRelation, joinType = outerJT, condition = Some('a === 'd)) + joinedRelation.join(testRelation, joinType = outerJT, + condition = Some("a".attr === "d".attr)) val optimized = Optimize.execute(originalQuery.analyze) val pushedDownJoin = - testRelation1.join(testRelation, joinType = outerJT, condition = Some('a === 'd)) + testRelation1.join(testRelation, joinType = outerJT, + condition = Some("a".attr === "d".attr)) val correctAnswer = pushedDownJoin.join(testRelation2, joinType = innerJT, innerJoinCond).analyze comparePlans(optimized, correctAnswer) @@ -371,17 +396,19 @@ class LeftSemiPushdownSuite extends PlanTest { } } - Seq(Some('e === 'd), None).foreach { case innerJoinCond => + Seq(Some("e".attr === "d".attr), None).foreach { case innerJoinCond => Seq(LeftSemi, LeftAnti).foreach { case outerJT => Seq(Inner, RightOuter, Cross).foreach { case innerJT => test(s"$outerJT pushdown to right of join type: $innerJT join condition $innerJoinCond") { val joinedRelation = testRelation1.join(testRelation2, joinType = innerJT, innerJoinCond) val originalQuery = - joinedRelation.join(testRelation, joinType = outerJT, condition = Some('a === 'e)) + joinedRelation.join(testRelation, joinType = outerJT, + condition = Some("a".attr === "e".attr)) val optimized = Optimize.execute(originalQuery.analyze) val pushedDownJoin = - testRelation2.join(testRelation, joinType = outerJT, condition = Some('a === 'e)) + testRelation2.join(testRelation, joinType = outerJT, + condition = Some("a".attr === "e".attr)) val correctAnswer = testRelation1.join(pushedDownJoin, joinType = innerJT, innerJoinCond).analyze comparePlans(optimized, correctAnswer) @@ -394,7 +421,7 @@ class LeftSemiPushdownSuite extends PlanTest { test(s"$jt no pushdown - join condition refers left leg - join type for RightOuter") { val joinedRelation = testRelation1.join(testRelation2, joinType = RightOuter, None) val originalQuery = - joinedRelation.join(testRelation, joinType = jt, condition = Some('a === 'd)) + joinedRelation.join(testRelation, joinType = jt, condition = Some("a".attr === "d".attr)) val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, originalQuery.analyze) } @@ -404,7 +431,7 @@ class LeftSemiPushdownSuite extends PlanTest { test(s"$jt no pushdown - join condition refers right leg - join type for LeftOuter") { val joinedRelation = testRelation1.join(testRelation2, joinType = LeftOuter, None) val originalQuery = - joinedRelation.join(testRelation, joinType = jt, condition = Some('a === 'e)) + joinedRelation.join(testRelation, joinType = jt, condition = Some("a".attr === "e".attr)) val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, originalQuery.analyze) } @@ -415,7 +442,8 @@ class LeftSemiPushdownSuite extends PlanTest { test(s"$outerJT no pushdown - join condition refers both leg - join type $innerJT") { val joinedRelation = testRelation1.join(testRelation2, joinType = innerJT, None) val originalQuery = joinedRelation - .join(testRelation, joinType = outerJT, condition = Some('a === 'd && 'a === 'e)) + .join(testRelation, joinType = outerJT, + condition = Some("a".attr === "d".attr && "a".attr === "e".attr)) val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, originalQuery.analyze) } @@ -427,7 +455,8 @@ class LeftSemiPushdownSuite extends PlanTest { test(s"$outerJT no pushdown - join condition refers none of the leg - join type $innerJT") { val joinedRelation = testRelation1.join(testRelation2, joinType = innerJT, None) val originalQuery = joinedRelation - .join(testRelation, joinType = outerJT, condition = Some('d + 'e === 'a)) + .join(testRelation, joinType = outerJT, + condition = Some("d".attr + "e".attr === "a".attr)) val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, originalQuery.analyze) } @@ -438,7 +467,7 @@ class LeftSemiPushdownSuite extends PlanTest { test(s"$jt no pushdown when child join type is FullOuter") { val joinedRelation = testRelation1.join(testRelation2, joinType = FullOuter, None) val originalQuery = - joinedRelation.join(testRelation, joinType = jt, condition = Some('a === 'e)) + joinedRelation.join(testRelation, joinType = jt, condition = Some("a".attr === "e".attr)) val optimized = Optimize.execute(originalQuery.analyze) comparePlans(optimized, originalQuery.analyze) } @@ -449,14 +478,14 @@ class LeftSemiPushdownSuite extends PlanTest { Seq(-1, 100000).foreach { threshold => withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> threshold.toString) { val originalQuery = testRelation - .groupBy('b)('b) - .join(testRelation1, joinType = jt, condition = Some('b <=> 'd)) + .groupBy("b".attr)("b".attr) + .join(testRelation1, joinType = jt, condition = Some("b".attr <=> "d".attr)) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = if (threshold > 0) { testRelation - .join(testRelation1, joinType = jt, condition = Some('b <=> 'd)) - .groupBy('b)('b) + .join(testRelation1, joinType = jt, condition = Some("b".attr <=> "d".attr)) + .groupBy("b".attr)("b".attr) .analyze } else { originalQuery.analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala index c06c92f9c151..b1d5fc94664e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala @@ -33,16 +33,16 @@ class LikeSimplificationSuite extends PlanTest { LikeSimplification) :: Nil } - val testRelation = LocalRelation('a.string) + val testRelation = LocalRelation("a".attr.string) test("simplify Like into StartsWith") { val originalQuery = testRelation - .where(('a like "abc%") || ('a like "abc\\%")) + .where(("a".attr like "abc%") || ("a".attr like "abc\\%")) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where(StartsWith('a, "abc") || ('a like "abc\\%")) + .where(StartsWith("a".attr, "abc") || ("a".attr like "abc\\%")) .analyze comparePlans(optimized, correctAnswer) @@ -51,11 +51,11 @@ class LikeSimplificationSuite extends PlanTest { test("simplify Like into EndsWith") { val originalQuery = testRelation - .where('a like "%xyz") + .where("a".attr like "%xyz") val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where(EndsWith('a, "xyz")) + .where(EndsWith("a".attr, "xyz")) .analyze comparePlans(optimized, correctAnswer) @@ -64,12 +64,12 @@ class LikeSimplificationSuite extends PlanTest { test("simplify Like into startsWith and EndsWith") { val originalQuery = testRelation - .where(('a like "abc\\%def") || ('a like "abc%def")) + .where(("a".attr like "abc\\%def") || ("a".attr like "abc%def")) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where(('a like "abc\\%def") || - (Length('a) >= 6 && (StartsWith('a, "abc") && EndsWith('a, "def")))) + .where(("a".attr like "abc\\%def") || + (Length("a".attr) >= 6 && (StartsWith("a".attr, "abc") && EndsWith("a".attr, "def")))) .analyze comparePlans(optimized, correctAnswer) @@ -78,11 +78,11 @@ class LikeSimplificationSuite extends PlanTest { test("simplify Like into Contains") { val originalQuery = testRelation - .where(('a like "%mn%") || ('a like "%mn\\%")) + .where(("a".attr like "%mn%") || ("a".attr like "%mn\\%")) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where(Contains('a, "mn") || ('a like "%mn\\%")) + .where(Contains("a".attr, "mn") || ("a".attr like "%mn\\%")) .analyze comparePlans(optimized, correctAnswer) @@ -91,28 +91,28 @@ class LikeSimplificationSuite extends PlanTest { test("simplify Like into EqualTo") { val originalQuery = testRelation - .where(('a like "") || ('a like "abc")) + .where(("a".attr like "") || ("a".attr like "abc")) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where(('a === "") || ('a === "abc")) + .where(("a".attr === "") || ("a".attr === "abc")) .analyze comparePlans(optimized, correctAnswer) } test("null pattern") { - val originalQuery = testRelation.where('a like Literal(null, StringType)).analyze + val originalQuery = testRelation.where("a".attr like Literal(null, StringType)).analyze val optimized = Optimize.execute(originalQuery) comparePlans(optimized, testRelation.where(Literal(null, BooleanType)).analyze) } test("test like escape syntax") { - val originalQuery1 = testRelation.where('a.like("abc#%", '#')) + val originalQuery1 = testRelation.where("a".attr.like("abc#%", '#')) val optimized1 = Optimize.execute(originalQuery1.analyze) comparePlans(optimized1, originalQuery1.analyze) - val originalQuery2 = testRelation.where('a.like("abc#%abc", '#')) + val originalQuery2 = testRelation.where("a".attr.like("abc#%abc", '#')) val optimized2 = Optimize.execute(originalQuery2.analyze) comparePlans(optimized2, originalQuery2.analyze) } @@ -120,47 +120,47 @@ class LikeSimplificationSuite extends PlanTest { test("SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") { val originalQuery1 = testRelation - .where(('a like "abc%") || ('a like "\\abc%")) + .where(("a".attr like "abc%") || ("a".attr like "\\abc%")) val optimized1 = Optimize.execute(originalQuery1.analyze) val correctAnswer1 = testRelation - .where(StartsWith('a, "abc") || ('a like "\\abc%")) + .where(StartsWith("a".attr, "abc") || ("a".attr like "\\abc%")) .analyze comparePlans(optimized1, correctAnswer1) val originalQuery2 = testRelation - .where(('a like "%xyz") || ('a like "%xyz\\")) + .where(("a".attr like "%xyz") || ("a".attr like "%xyz\\")) val optimized2 = Optimize.execute(originalQuery2.analyze) val correctAnswer2 = testRelation - .where(EndsWith('a, "xyz") || ('a like "%xyz\\")) + .where(EndsWith("a".attr, "xyz") || ("a".attr like "%xyz\\")) .analyze comparePlans(optimized2, correctAnswer2) val originalQuery3 = testRelation - .where(('a like ("@bc%def", '@')) || ('a like "abc%def")) + .where(("a".attr like ("@bc%def", '@')) || ("a".attr like "abc%def")) val optimized3 = Optimize.execute(originalQuery3.analyze) val correctAnswer3 = testRelation - .where(('a like ("@bc%def", '@')) || - (Length('a) >= 6 && (StartsWith('a, "abc") && EndsWith('a, "def")))) + .where(("a".attr like ("@bc%def", '@')) || + (Length("a".attr) >= 6 && (StartsWith("a".attr, "abc") && EndsWith("a".attr, "def")))) .analyze comparePlans(optimized3, correctAnswer3) val originalQuery4 = testRelation - .where(('a like "%mn%") || ('a like ("%mn%", '%'))) + .where(("a".attr like "%mn%") || ("a".attr like ("%mn%", '%'))) val optimized4 = Optimize.execute(originalQuery4.analyze) val correctAnswer4 = testRelation - .where(Contains('a, "mn") || ('a like ("%mn%", '%'))) + .where(Contains("a".attr, "mn") || ("a".attr like ("%mn%", '%'))) .analyze comparePlans(optimized4, correctAnswer4) val originalQuery5 = testRelation - .where(('a like "abc") || ('a like ("abbc", 'b'))) + .where(("a".attr like "abc") || ("a".attr like ("abbc", 'b'))) val optimized5 = Optimize.execute(originalQuery5.analyze) val correctAnswer5 = testRelation - .where(('a === "abc") || ('a like ("abbc", 'b'))) + .where(("a".attr === "abc") || ("a".attr like ("abbc", 'b'))) .analyze comparePlans(optimized5, correctAnswer5) } @@ -168,15 +168,15 @@ class LikeSimplificationSuite extends PlanTest { test("simplify LikeAll") { val originalQuery = testRelation - .where(('a likeAll( + .where(("a".attr likeAll( "abc%", "abc\\%", "%xyz", "abc\\%def", "abc%def", "%mn%", "%mn\\%", "", "abc"))) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where((((((StartsWith('a, "abc") && EndsWith('a, "xyz")) && - (Length('a) >= 6 && (StartsWith('a, "abc") && EndsWith('a, "def")))) && - Contains('a, "mn")) && ('a === "")) && ('a === "abc")) && - ('a likeAll("abc\\%", "abc\\%def", "%mn\\%"))) + .where((((((StartsWith("a".attr, "abc") && EndsWith("a".attr, "xyz")) && + (Length("a".attr) >= 6 && (StartsWith("a".attr, "abc") && EndsWith("a".attr, "def")))) && + Contains("a".attr, "mn")) && ("a".attr === "")) && ("a".attr === "abc")) && + ("a".attr likeAll("abc\\%", "abc\\%def", "%mn\\%"))) .analyze comparePlans(optimized, correctAnswer) @@ -185,15 +185,15 @@ class LikeSimplificationSuite extends PlanTest { test("simplify NotLikeAll") { val originalQuery = testRelation - .where(('a notLikeAll( + .where(("a".attr notLikeAll( "abc%", "abc\\%", "%xyz", "abc\\%def", "abc%def", "%mn%", "%mn\\%", "", "abc"))) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where((((((Not(StartsWith('a, "abc")) && Not(EndsWith('a, "xyz"))) && - Not(Length('a) >= 6 && (StartsWith('a, "abc") && EndsWith('a, "def")))) && - Not(Contains('a, "mn"))) && Not('a === "")) && Not('a === "abc")) && - ('a notLikeAll("abc\\%", "abc\\%def", "%mn\\%"))) + .where((((((Not(StartsWith("a".attr, "abc")) && Not(EndsWith("a".attr, "xyz"))) && + Not(Length("a".attr) >= 6 && (StartsWith("a".attr, "abc") && EndsWith("a".attr, "def")))) && + Not(Contains("a".attr, "mn"))) && Not("a".attr === "")) && Not("a".attr === "abc")) && + ("a".attr notLikeAll("abc\\%", "abc\\%def", "%mn\\%"))) .analyze comparePlans(optimized, correctAnswer) @@ -202,15 +202,15 @@ class LikeSimplificationSuite extends PlanTest { test("simplify LikeAny") { val originalQuery = testRelation - .where(('a likeAny( + .where(("a".attr likeAny( "abc%", "abc\\%", "%xyz", "abc\\%def", "abc%def", "%mn%", "%mn\\%", "", "abc"))) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where((((((StartsWith('a, "abc") || EndsWith('a, "xyz")) || - (Length('a) >= 6 && (StartsWith('a, "abc") && EndsWith('a, "def")))) || - Contains('a, "mn")) || ('a === "")) || ('a === "abc")) || - ('a likeAny("abc\\%", "abc\\%def", "%mn\\%"))) + .where((((((StartsWith("a".attr, "abc") || EndsWith("a".attr, "xyz")) || + (Length("a".attr) >= 6 && (StartsWith("a".attr, "abc") && EndsWith("a".attr, "def")))) || + Contains("a".attr, "mn")) || ("a".attr === "")) || ("a".attr === "abc")) || + ("a".attr likeAny("abc\\%", "abc\\%def", "%mn\\%"))) .analyze comparePlans(optimized, correctAnswer) @@ -219,15 +219,15 @@ class LikeSimplificationSuite extends PlanTest { test("simplify NotLikeAny") { val originalQuery = testRelation - .where(('a notLikeAny( + .where(("a".attr notLikeAny( "abc%", "abc\\%", "%xyz", "abc\\%def", "abc%def", "%mn%", "%mn\\%", "", "abc"))) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where((((((Not(StartsWith('a, "abc")) || Not(EndsWith('a, "xyz"))) || - Not(Length('a) >= 6 && (StartsWith('a, "abc") && EndsWith('a, "def")))) || - Not(Contains('a, "mn"))) || Not('a === "")) || Not('a === "abc")) || - ('a notLikeAny("abc\\%", "abc\\%def", "%mn\\%"))) + .where((((((Not(StartsWith("a".attr, "abc")) || Not(EndsWith("a".attr, "xyz"))) || + Not(Length("a".attr) >= 6 && (StartsWith("a".attr, "abc") && EndsWith("a".attr, "def")))) || + Not(Contains("a".attr, "mn"))) || Not("a".attr === "")) || Not("a".attr === "abc")) || + ("a".attr notLikeAny("abc\\%", "abc\\%def", "%mn\\%"))) .analyze comparePlans(optimized, correctAnswer) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala index 7a33b5b4b53d..52ab6b32cf12 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala @@ -45,8 +45,8 @@ class LimitPushdownSuite extends PlanTest { private val testRelation2 = LocalRelation.fromExternalRows( Seq("d".attr.int, "e".attr.int, "f".attr.int), 1.to(6).map(_ => Row(1, 2, 3))) - private val x = testRelation.subquery('x) - private val y = testRelation.subquery('y) + private val x = testRelation.subquery("x") + private val y = testRelation.subquery("y") // Union --------------------------------------------------------------------------------------- @@ -75,21 +75,21 @@ class LimitPushdownSuite extends PlanTest { } test("Union: no limit to both sides if children having smaller limit values") { - val unionQuery = - Union(testRelation.limit(1), testRelation2.select('d, 'e, 'f).limit(1)).limit(2) + val unionQuery = Union(testRelation.limit(1), + testRelation2.select("d".attr, "e".attr, "f".attr).limit(1)).limit(2) val unionOptimized = Optimize.execute(unionQuery.analyze) - val unionCorrectAnswer = - Union(testRelation.limit(1), testRelation2.select('d, 'e, 'f).limit(1)).analyze + val unionCorrectAnswer = Union(testRelation.limit(1), + testRelation2.select("d".attr, "e".attr, "f".attr).limit(1)).analyze comparePlans(unionOptimized, unionCorrectAnswer) } test("Union: limit to each sides if children having larger limit values") { - val unionQuery = - Union(testRelation.limit(3), testRelation2.select('d, 'e, 'f).limit(4)).limit(2) + val unionQuery = Union(testRelation.limit(3), + testRelation2.select("d".attr, "e".attr, "f".attr).limit(4)).limit(2) val unionOptimized = Optimize.execute(unionQuery.analyze) val unionCorrectAnswer = - Limit(2, Union( - LocalLimit(2, testRelation), LocalLimit(2, testRelation2.select('d, 'e, 'f)))).analyze + Limit(2, Union(LocalLimit(2, testRelation), + LocalLimit(2, testRelation2.select("d".attr, "e".attr, "f".attr)))).analyze comparePlans(unionOptimized, unionCorrectAnswer) } @@ -153,7 +153,7 @@ class LimitPushdownSuite extends PlanTest { } test("full outer join where neither side is limited and left side has larger statistics") { - val xBig = testRelation.copy(data = Seq.fill(10)(null)).subquery('x) + val xBig = testRelation.copy(data = Seq.fill(10)(null)).subquery("x") assert(xBig.stats.sizeInBytes > y.stats.sizeInBytes) val originalQuery = xBig.join(y, FullOuter).limit(1).analyze val optimized = Optimize.execute(originalQuery) @@ -162,7 +162,7 @@ class LimitPushdownSuite extends PlanTest { } test("full outer join where neither side is limited and right side has larger statistics") { - val yBig = testRelation.copy(data = Seq.fill(10)(null)).subquery('y) + val yBig = testRelation.copy(data = Seq.fill(10)(null)).subquery("y") assert(x.stats.sizeInBytes < yBig.stats.sizeInBytes) val originalQuery = x.join(yBig, FullOuter).limit(1).analyze val optimized = Optimize.execute(originalQuery) @@ -186,12 +186,12 @@ class LimitPushdownSuite extends PlanTest { // test push down val analyzed2 = Limit(1, Union( - x.groupBy(Symbol("a"))(count(1)), - y.groupBy(Symbol("b"))(count(1)))).analyze + x.groupBy("a".attr)(count(1)), + y.groupBy("b".attr)(count(1)))).analyze val optimized2 = Optimize.execute(analyzed2) val expected2 = Limit(1, Union( - LocalLimit(1, x.groupBy(Symbol("a"))(count(1))), - LocalLimit(1, y.groupBy(Symbol("b"))(count(1))))).analyze + LocalLimit(1, x.groupBy("a".attr)(count(1))), + LocalLimit(1, y.groupBy("b".attr)(count(1))))).analyze comparePlans(expected2, optimized2) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala index c83ab375ee15..c5b9751e25c3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala @@ -41,16 +41,16 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { private val name = StructType.fromDDL("first string, middle string, last string") private val employer = StructType.fromDDL("id int, company struct") private val contact = LocalRelation( - 'id.int, - 'name.struct(name), - 'address.string, - 'friends.array(name), - 'relatives.map(StringType, name), - 'employer.struct(employer)) + "id".attr.int, + "name".attr.struct(name), + "address".attr.string, + "friends".attr.array(name), + "relatives".attr.mapAttr(StringType, name), + "employer".attr.struct(employer)) test("Pushing a single nested field projection") { def testSingleFieldPushDown(op: LogicalPlan => LogicalPlan): Unit = { - val middle = GetStructField('name, 1, Some("middle")) + val middle = GetStructField("name".attr, 1, Some("middle")) val query = op(contact).select(middle).analyze val optimized = Optimize.execute(query) val expected = op(contact.select(middle)).analyze @@ -63,18 +63,18 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { } test("Pushing multiple nested field projection") { - val first = GetStructField('name, 0, Some("first")) - val last = GetStructField('name, 2, Some("last")) + val first = GetStructField("name".attr, 0, Some("first")) + val last = GetStructField("name".attr, 2, Some("last")) val query = contact .limit(5) - .select('id, first, last) + .select("id".attr, first, last) .analyze val optimized = Optimize.execute(query) val expected = contact - .select('id, first, last) + .select("id".attr, first, last) .limit(5) .analyze @@ -82,12 +82,12 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { } test("function with nested field inputs") { - val first = GetStructField('name, 0, Some("first")) - val last = GetStructField('name, 2, Some("last")) + val first = GetStructField("name".attr, 0, Some("first")) + val last = GetStructField("name".attr, 2, Some("last")) val query = contact .limit(5) - .select('id, ConcatWs(Seq(first, last))) + .select("id".attr, ConcatWs(Seq(first, last))) .analyze val optimized = Optimize.execute(query) @@ -95,18 +95,19 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { val aliases = collectGeneratedAliases(optimized) val expected = contact - .select('id, first.as(aliases(0)), last.as(aliases(1))) + .select("id".attr, first.as(aliases(0)), last.as(aliases(1))) .limit(5) .select( - 'id, + "id".attr, ConcatWs(Seq($"${aliases(0)}", $"${aliases(1)}")).as("concat_ws(name.first, name.last)")) .analyze comparePlans(optimized, expected) } test("multi-level nested field") { - val field1 = GetStructField(GetStructField('employer, 1, Some("company")), 0, Some("name")) - val field2 = GetStructField('employer, 0, Some("id")) + val field1 = + GetStructField(GetStructField("employer".attr, 1, Some("company")), 0, Some("name")) + val field2 = GetStructField("employer".attr, 0, Some("id")) val query = contact .limit(5) @@ -123,18 +124,18 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { } test("Push original case-sensitive names") { - val first1 = GetStructField('name, 0, Some("first")) - val first2 = GetStructField('name, 1, Some("FIRST")) + val first1 = GetStructField("name".attr, 0, Some("first")) + val first2 = GetStructField("name".attr, 1, Some("FIRST")) val query = contact .limit(5) - .select('id, first1, first2) + .select("id".attr, first1, first2) .analyze val optimized = Optimize.execute(query) val expected = contact - .select('id, first1, first2) + .select("id".attr, first1, first2) .limit(5) .analyze @@ -143,15 +144,15 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { test("Pushing a single nested field projection - negative") { val ops = Seq( - (input: LogicalPlan) => input.distribute('name)(1), - (input: LogicalPlan) => input.orderBy('name.asc), - (input: LogicalPlan) => input.sortBy('name.asc), + (input: LogicalPlan) => input.distribute("name".attr)(1), + (input: LogicalPlan) => input.orderBy("name".attr.asc), + (input: LogicalPlan) => input.sortBy("name".attr.asc), (input: LogicalPlan) => input.union(input) ) val queries = ops.map { op => - op(contact.select('name)) - .select(GetStructField('name, 1, Some("middle"))) + op(contact.select("name".attr)) + .select(GetStructField("name".attr, 1, Some("middle"))) .analyze } @@ -161,20 +162,20 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { comparePlans(optimized, expected) } val expectedUnion = - contact.select('name).union(contact.select('name.as('name))) - .select(GetStructField('name, 1, Some("middle"))).analyze + contact.select("name".attr).union(contact.select("name".attr.as("name"))) + .select(GetStructField("name".attr, 1, Some("middle"))).analyze comparePlans(optimizedUnion, expectedUnion) } test("Pushing a single nested field projection through filters - negative") { val ops = Array( - (input: LogicalPlan) => input.where('name.isNotNull), + (input: LogicalPlan) => input.where("name".attr.isNotNull), (input: LogicalPlan) => input.where($"name.middle".isNotNull) ) val queries = ops.map { op => op(contact) - .select(GetStructField('name, 1, Some("middle"))) + .select(GetStructField("name".attr, 1, Some("middle"))) .analyze } @@ -189,25 +190,26 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { test("Do not optimize when parent field is used") { val query = contact .limit(5) - .select('id, GetStructField('name, 0, Some("first")), 'name) + .select("id".attr, GetStructField("name".attr, 0, Some("first")), "name".attr) .analyze val optimized = Optimize.execute(query) val expected = contact - .select('id, 'name) + .select("id".attr, "name".attr) .limit(5) - .select('id, GetStructField('name, 0, Some("first")), 'name) + .select("id".attr, GetStructField("name".attr, 0, Some("first")), "name".attr) .analyze comparePlans(optimized, expected) } test("Some nested column means the whole structure") { - val nestedRelation = LocalRelation('a.struct('b.struct('c.int, 'd.int, 'e.int))) + val nestedRelation = LocalRelation("a".attr.struct("b".attr.struct("c".attr.int, + "d".attr.int, "e".attr.int))) val query = nestedRelation .limit(5) - .select(GetStructField('a, 0, Some("b"))) + .select(GetStructField("a".attr, 0, Some("b"))) .analyze val optimized = Optimize.execute(query) @@ -216,12 +218,12 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { } test("nested field pruning for getting struct field in array of struct") { - val field1 = GetArrayStructFields(child = 'friends, + val field1 = GetArrayStructFields(child = "friends".attr, field = StructField("first", StringType), ordinal = 0, numFields = 3, containsNull = true) - val field2 = GetStructField('employer, 0, Some("id")) + val field2 = GetStructField("employer".attr, 0, Some("id")) val query = contact .limit(5) @@ -238,8 +240,8 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { } test("nested field pruning for getting struct field in map") { - val field1 = GetStructField(GetMapValue('relatives, Literal("key")), 0, Some("first")) - val field2 = GetArrayStructFields(child = MapValues('relatives), + val field1 = GetStructField(GetMapValue("relatives".attr, Literal("key")), 0, Some("first")) + val field2 = GetArrayStructFields(child = MapValues("relatives".attr), field = StructField("middle", StringType), ordinal = 1, numFields = 3, @@ -260,15 +262,15 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { } test("SPARK-27633: Do not generate redundant aliases if parent nested field is aliased too") { - val nestedRelation = LocalRelation('a.struct('b.struct('c.int, - 'd.struct('f.int, 'g.int)), 'e.int)) + val nestedRelation = LocalRelation("a".attr.struct("b".attr.struct("c".attr.int, + "d".attr.struct("f".attr.int, "g".attr.int)), "e".attr.int)) // `a.b` - val first = 'a.getField("b") + val first = "a".attr.getField("b") // `a.b.c` + 1 - val second = 'a.getField("b").getField("c") + Literal(1) + val second = "a".attr.getField("b").getField("c") + Literal(1) // `a.b.d.f` - val last = 'a.getField("b").getField("d").getField("f") + val last = "a".attr.getField("b").getField("d").getField("f") val query = nestedRelation .limit(5) @@ -292,8 +294,8 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { test("Nested field pruning for Project and Generate") { val query = contact - .generate(Explode('friends.getField("first")), outputNames = Seq("explode")) - .select('explode, 'friends.getField("middle")) + .generate(Explode("friends".attr.getField("first")), outputNames = Seq("explode")) + .select("explode".attr, "friends".attr.getField("middle")) .analyze val optimized = Optimize.execute(query) @@ -301,27 +303,27 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { val expected = contact .select( - 'friends.getField("middle").as(aliases(0)), - 'friends.getField("first").as(aliases(1))) + "friends".attr.getField("middle").as(aliases(0)), + "friends".attr.getField("first").as(aliases(1))) .generate(Explode($"${aliases(1)}"), unrequiredChildIndex = Seq(1), outputNames = Seq("explode")) - .select('explode, $"${aliases(0)}".as("friends.middle")) + .select("explode".attr, $"${aliases(0)}".as("friends.middle")) .analyze comparePlans(optimized, expected) } test("Nested field pruning for Generate") { val query = contact - .generate(Explode('friends.getField("first")), outputNames = Seq("explode")) - .select('explode) + .generate(Explode("friends".attr.getField("first")), outputNames = Seq("explode")) + .select("explode".attr) .analyze val optimized = Optimize.execute(query) val aliases = collectGeneratedAliases(optimized) val expected = contact - .select('friends.getField("first").as(aliases(0))) + .select("friends".attr.getField("first").as(aliases(0))) .generate(Explode($"${aliases(0)}"), unrequiredChildIndex = Seq(0), outputNames = Seq("explode")) @@ -331,23 +333,23 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { test("Nested field pruning for Project and Generate: not prune on generator output") { val companies = LocalRelation( - 'id.int, - 'employers.array(employer)) + "id".attr.int, + "employers".attr.array(employer)) val query = companies - .generate(Explode('employers.getField("company")), outputNames = Seq("company")) - .select('company.getField("name")) + .generate(Explode("employers".attr.getField("company")), outputNames = Seq("company")) + .select("company".attr.getField("name")) .analyze val optimized = Optimize.execute(query) val aliases = collectGeneratedAliases(optimized) val expected = companies - .select('employers.getField("company").as(aliases(0))) + .select("employers".attr.getField("company").as(aliases(0))) .generate(Explode($"${aliases(0)}"), unrequiredChildIndex = Seq(0), outputNames = Seq("company")) - .select('company.getField("name").as("company.name")) + .select("company".attr.getField("name").as("company.name")) .analyze comparePlans(optimized, expected) } @@ -355,17 +357,17 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { test("Nested field pruning for Generate: not prune on required child output") { val query = contact .generate( - Explode('friends.getField("first")), + Explode("friends".attr.getField("first")), outputNames = Seq("explode")) - .select('explode, 'friends) + .select("explode".attr, "friends".attr) .analyze val optimized = Optimize.execute(query) val expected = contact - .select('friends) - .generate(Explode('friends.getField("first")), + .select("friends".attr) + .generate(Explode("friends".attr.getField("first")), outputNames = Seq("explode")) - .select('explode, 'friends) + .select("explode".attr, "friends".attr) .analyze comparePlans(optimized, expected) } @@ -380,7 +382,7 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { val aliases1 = collectGeneratedAliases(optimized1) val expected1 = contact - .select('id, 'name.getField("middle").as(aliases1(0))) + .select("id".attr, "name".attr.getField("middle").as(aliases1(0))) .distribute($"id")(1) .select($"${aliases1(0)}".as("middle")) .analyze @@ -395,7 +397,7 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { val aliases2 = collectGeneratedAliases(optimized2) val expected2 = contact - .select('name.getField("middle").as(aliases2(0))) + .select("name".attr.getField("middle").as(aliases2(0))) .distribute($"${aliases2(0)}")(1) .select($"${aliases2(0)}".as("middle")) .analyze @@ -413,8 +415,8 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { test("Nested field pruning through Join") { val department = LocalRelation( - 'depID.int, - 'personID.string) + "depID".attr.int, + "personID".attr.string) val query1 = contact.join(department, condition = Some($"id" === $"depID")) .select($"name.middle") @@ -423,8 +425,8 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { val aliases1 = collectGeneratedAliases(optimized1) - val expected1 = contact.select('id, 'name.getField("middle").as(aliases1(0))) - .join(department.select('depID), condition = Some($"id" === $"depID")) + val expected1 = contact.select("id".attr, "name".attr.getField("middle").as(aliases1(0))) + .join(department.select("depID".attr), condition = Some($"id" === $"depID")) .select($"${aliases1(0)}".as("middle")) .analyze comparePlans(optimized1, expected1) @@ -437,15 +439,16 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { val aliases2 = collectGeneratedAliases(optimized2) val expected2 = contact.select( - 'name.getField("first").as(aliases2(0)), - 'name.getField("middle").as(aliases2(1))) - .join(department.select('personID), condition = Some($"${aliases2(1)}" === $"personID")) + "name".attr.getField("first").as(aliases2(0)), + "name".attr.getField("middle").as(aliases2(1))) + .join(department.select("personID".attr), + condition = Some($"${aliases2(1)}" === $"personID")) .select($"${aliases2(0)}".as("first")) .analyze comparePlans(optimized2, expected2) - val contact2 = LocalRelation('name2.struct(name)) - val query3 = contact.select('name) + val contact2 = LocalRelation("name2".attr.struct(name)) + val query3 = contact.select("name".attr) .join(contact2, condition = Some($"name" === $"name2")) .select($"name.first") .analyze @@ -461,7 +464,7 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { val expected1 = basePlan( contact - .select($"id", 'name.getField("first").as(aliases1(0))) + .select($"id", "name".attr.getField("first").as(aliases1(0))) ).groupBy($"id")(first($"${aliases1(0)}").as("first")).analyze comparePlans(optimized1, expected1) @@ -471,7 +474,8 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { val expected2 = basePlan( contact - .select('name.getField("last").as(aliases2(0)), 'name.getField("first").as(aliases2(1))) + .select("name".attr.getField("last").as(aliases2(0)), + "name".attr.getField("first").as(aliases2(1))) ).groupBy($"${aliases2(0)}")(first($"${aliases2(1)}").as("first")).analyze comparePlans(optimized2, expected2) } @@ -495,7 +499,7 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { val spec = windowSpec($"address" :: Nil, $"id".asc :: Nil, UnspecifiedFrame) val winExpr = windowExpr(RowNumber(), spec) val query = contact - .select($"name.first", winExpr.as('window)) + .select($"name.first", winExpr.as("window")) .orderBy($"name.last".asc) .analyze val optimized = Optimize.execute(query) @@ -513,7 +517,7 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { test("Nested field pruning for Filter with other supported operators") { val spec = windowSpec($"address" :: Nil, $"id".asc :: Nil, UnspecifiedFrame) val winExpr = windowExpr(RowNumber(), spec) - val query1 = contact.select($"name.first", winExpr.as('window)) + val query1 = contact.select($"name.first", winExpr.as("window")) .where($"window" === 1 && $"name.first" === "a") .analyze val optimized1 = Optimize.execute(query1) @@ -558,8 +562,8 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { comparePlans(optimized3, expected3) val department = LocalRelation( - 'depID.int, - 'personID.string) + "depID".attr.int, + "personID".attr.string) val query4 = contact.join(department, condition = Some($"id" === $"depID")) .where($"name.first" === "a") .select($"name.first") @@ -568,7 +572,7 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { val aliases4 = collectGeneratedAliases(optimized4) val expected4 = contact .select($"id", $"name.first".as(aliases4(1))) - .join(department.select('depID), condition = Some($"id" === $"depID")) + .join(department.select("depID".attr), condition = Some($"id" === $"depID")) .select($"${aliases4(1)}".as(aliases4(0))) .where($"${aliases4(0)}" === "a") .select($"${aliases4(0)}".as("first")) @@ -637,7 +641,7 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { Seq(ConcatWs(Seq($"name.first", $"name.middle")), ConcatWs(Seq($"name.middle", $"name.first"))) ), - Seq('a.string, 'b.string), + Seq("a".attr.string, "b".attr.string), basePlan(contact) ).analyze val optimized1 = Optimize.execute(query1) @@ -649,10 +653,10 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { Seq(ConcatWs(Seq($"${aliases1(0)}", $"${aliases1(1)}")), ConcatWs(Seq($"${aliases1(1)}", $"${aliases1(0)}"))) ), - Seq('a.string, 'b.string), + Seq("a".attr.string, "b".attr.string), basePlan(contact.select( - 'name.getField("first").as(aliases1(0)), - 'name.getField("middle").as(aliases1(1)))) + "name".attr.getField("first").as(aliases1(0)), + "name".attr.getField("middle").as(aliases1(1)))) ).analyze comparePlans(optimized1, expected1) } @@ -670,7 +674,7 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { Seq($"name", $"name.middle"), Seq($"name", ConcatWs(Seq($"name.middle", $"name.first"))) ), - Seq('a.string, 'b.string), + Seq("a".attr.string, "b".attr.string), contact ).analyze val optimized2 = Optimize.execute(query2) @@ -679,7 +683,7 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { Seq($"name", $"name.middle"), Seq($"name", ConcatWs(Seq($"name.middle", $"name.first"))) ), - Seq('a.string, 'b.string), + Seq("a".attr.string, "b".attr.string), contact.select($"name") ).analyze comparePlans(optimized2, expected2) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala index bb9919f94eef..c6b0bbf0a7df 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala @@ -30,9 +30,9 @@ class NormalizeFloatingPointNumbersSuite extends PlanTest { val batches = Batch("NormalizeFloatingPointNumbers", Once, NormalizeFloatingNumbers) :: Nil } - val testRelation1 = LocalRelation('a.double) + val testRelation1 = LocalRelation("a".attr.double) val a = testRelation1.output(0) - val testRelation2 = LocalRelation('a.double) + val testRelation2 = LocalRelation("a".attr.double) val b = testRelation2.output(0) test("normalize floating points in window function expressions") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ObjectSerializerPruningSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ObjectSerializerPruningSuite.scala index 6d7c4c3c7e9d..2a906f542f84 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ObjectSerializerPruningSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ObjectSerializerPruningSuite.scala @@ -73,10 +73,10 @@ class ObjectSerializerPruningSuite extends PlanTest { } test("SPARK-26619: Prune the unused serializers from SerializeFromObject") { - val testRelation = LocalRelation('_1.int, '_2.int) + val testRelation = LocalRelation("_1".attr.int, "_2".attr.int) val serializerObject = CatalystSerde.serialize[(Int, Int)]( CatalystSerde.deserialize[(Int, Int)](testRelation)) - val query = serializerObject.select('_1) + val query = serializerObject.select("_1".attr) val optimized = Optimize.execute(query.analyze) val expected = serializerObject.copy(serializer = Seq(serializerObject.serializer.head)).analyze comparePlans(optimized, expected) @@ -84,7 +84,8 @@ class ObjectSerializerPruningSuite extends PlanTest { test("Prune nested serializers") { withSQLConf(SQLConf.SERIALIZER_NESTED_SCHEMA_PRUNING_ENABLED.key -> "true") { - val testRelation = LocalRelation('_1.struct(StructType.fromDDL("_1 int, _2 string")), '_2.int) + val testRelation = + LocalRelation("_1".attr.struct(StructType.fromDDL("_1 int, _2 string")), "_2".attr.int) val serializerObject = CatalystSerde.serialize[((Int, String), Int)]( CatalystSerde.deserialize[((Int, String), Int)](testRelation)) val query = serializerObject.select($"_1._1") @@ -111,7 +112,8 @@ class ObjectSerializerPruningSuite extends PlanTest { test("SPARK-32652: Prune nested serializers: RowEncoder") { withSQLConf(SQLConf.SERIALIZER_NESTED_SCHEMA_PRUNING_ENABLED.key -> "true") { - val testRelation = LocalRelation('i.struct(StructType.fromDDL("a int, b string")), 'j.int) + val testRelation = + LocalRelation("i".attr.struct(StructType.fromDDL("a int, b string")), "j".attr.int) val rowEncoder = RowEncoder(new StructType() .add("i", new StructType().add("a", "int").add("b", "string")) .add("j", "int")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeCsvExprsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeCsvExprsSuite.scala index 9b208cf2b57c..eafc6d88fb4b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeCsvExprsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeCsvExprsSuite.scala @@ -43,29 +43,31 @@ class OptimizeCsvExprsSuite extends PlanTest with ExpressionEvalHelper { val schema = StructType.fromDDL("a int, b int") - private val csvAttr = 'csv.string + private val csvAttr = "csv".attr.string private val testRelation = LocalRelation(csvAttr) test("SPARK-32968: prune unnecessary columns from GetStructField + from_csv") { val options = Map.empty[String, String] val query1 = testRelation - .select(GetStructField(CsvToStructs(schema, options, 'csv), 0)) + .select(GetStructField(CsvToStructs(schema, options, "csv".attr), 0)) val optimized1 = Optimizer.execute(query1.analyze) val prunedSchema1 = StructType.fromDDL("a int") val expected1 = testRelation - .select(GetStructField(CsvToStructs(schema, options, 'csv, None, Some(prunedSchema1)), 0)) + .select(GetStructField( + CsvToStructs(schema, options, "csv".attr, None, Some(prunedSchema1)), 0)) .analyze comparePlans(optimized1, expected1) val query2 = testRelation - .select(GetStructField(CsvToStructs(schema, options, 'csv), 1)) + .select(GetStructField(CsvToStructs(schema, options, "csv".attr), 1)) val optimized2 = Optimizer.execute(query2.analyze) val prunedSchema2 = StructType.fromDDL("b int") val expected2 = testRelation - .select(GetStructField(CsvToStructs(schema, options, 'csv, None, Some(prunedSchema2)), 0)) + .select(GetStructField( + CsvToStructs(schema, options, "csv".attr, None, Some(prunedSchema2)), 0)) .analyze comparePlans(optimized2, expected2) } @@ -74,7 +76,7 @@ class OptimizeCsvExprsSuite extends PlanTest with ExpressionEvalHelper { val options = Map("mode" -> "failfast") val query = testRelation - .select(GetStructField(CsvToStructs(schema, options, 'csv), 0)) + .select(GetStructField(CsvToStructs(schema, options, "csv".attr), 0)) val optimized = Optimizer.execute(query.analyze) val expected = query.analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala index a36083b84704..438f6db27a6e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala @@ -40,7 +40,7 @@ class OptimizeInSuite extends PlanTest { OptimizeIn) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) test("OptimizedIn test: Remove deterministic repetitions") { val originalQuery = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprsSuite.scala index ccbc61e8a498..059bd45f06b8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeJsonExprsSuite.scala @@ -44,8 +44,8 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { val schema = StructType.fromDDL("a int, b int") - private val structAtt = 'struct.struct(schema).notNull - private val jsonAttr = 'json.string + private val structAtt = "struct".attr.struct(schema).notNull + private val jsonAttr = "json".attr.string private val testRelation = LocalRelation(structAtt) private val testRelation2 = LocalRelation(jsonAttr) @@ -54,10 +54,10 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { val options = Map.empty[String, String] val query1 = testRelation - .select(JsonToStructs(schema, options, StructsToJson(options, 'struct)).as("struct")) + .select(JsonToStructs(schema, options, StructsToJson(options, "struct".attr)).as("struct")) val optimized1 = Optimizer.execute(query1.analyze) - val expected = testRelation.select('struct.as("struct")).analyze + val expected = testRelation.select("struct".attr.as("struct")).analyze comparePlans(optimized1, expected) val query2 = testRelation @@ -65,7 +65,7 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { JsonToStructs(schema, options, StructsToJson(options, JsonToStructs(schema, options, - StructsToJson(options, 'struct)))).as("struct")) + StructsToJson(options, "struct".attr)))).as("struct")) val optimized2 = Optimizer.execute(query2.analyze) comparePlans(optimized2, expected) @@ -76,11 +76,11 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { val schema = StructType.fromDDL("a int") val query = testRelation - .select(JsonToStructs(schema, options, StructsToJson(options, 'struct)).as("struct")) + .select(JsonToStructs(schema, options, StructsToJson(options, "struct".attr)).as("struct")) val optimized = Optimizer.execute(query.analyze) val expected = testRelation.select( - JsonToStructs(schema, options, StructsToJson(options, 'struct)).as("struct")).analyze + JsonToStructs(schema, options, StructsToJson(options, "struct".attr)).as("struct")).analyze comparePlans(optimized, expected) } @@ -90,11 +90,11 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { val schema = StructType.fromDDL("a int, B int") val query = testRelation - .select(JsonToStructs(schema, options, StructsToJson(options, 'struct)).as("struct")) + .select(JsonToStructs(schema, options, StructsToJson(options, "struct".attr)).as("struct")) val optimized = Optimizer.execute(query.analyze) val expected = testRelation.select( - JsonToStructs(schema, options, StructsToJson(options, 'struct)).as("struct")).analyze + JsonToStructs(schema, options, StructsToJson(options, "struct".attr)).as("struct")).analyze comparePlans(optimized, expected) } } @@ -104,17 +104,17 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { val nonNullSchema = StructType( StructField("a", IntegerType, false) :: StructField("b", IntegerType, false) :: Nil) - val structAtt = 'struct.struct(nonNullSchema).notNull + val structAtt = "struct".attr.struct(nonNullSchema).notNull val testRelationWithNonNullAttr = LocalRelation(structAtt) val schema = StructType.fromDDL("a int, b int") val query = testRelationWithNonNullAttr - .select(JsonToStructs(schema, options, StructsToJson(options, 'struct)).as("struct")) + .select(JsonToStructs(schema, options, StructsToJson(options, "struct".attr)).as("struct")) val optimized = Optimizer.execute(query.analyze) val expected = testRelationWithNonNullAttr.select( - JsonToStructs(schema, options, StructsToJson(options, 'struct)).as("struct")).analyze + JsonToStructs(schema, options, StructsToJson(options, "struct".attr)).as("struct")).analyze comparePlans(optimized, expected) } @@ -122,11 +122,11 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { val options = Map("testOption" -> "test") val query = testRelation - .select(JsonToStructs(schema, options, StructsToJson(options, 'struct)).as("struct")) + .select(JsonToStructs(schema, options, StructsToJson(options, "struct".attr)).as("struct")) val optimized = Optimizer.execute(query.analyze) val expected = testRelation.select( - JsonToStructs(schema, options, StructsToJson(options, 'struct)).as("struct")).analyze + JsonToStructs(schema, options, StructsToJson(options, "struct".attr)).as("struct")).analyze comparePlans(optimized, expected) } @@ -137,19 +137,19 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { val query1 = testRelation .select(JsonToStructs(schema, options, - StructsToJson(options, 'struct, Option(PST.getId)), UTC_OPT).as("struct")) + StructsToJson(options, "struct".attr, Option(PST.getId)), UTC_OPT).as("struct")) val optimized1 = Optimizer.execute(query1.analyze) val expected1 = testRelation.select( JsonToStructs(schema, options, - StructsToJson(options, 'struct, Option(PST.getId)), UTC_OPT).as("struct")).analyze + StructsToJson(options, "struct".attr, Option(PST.getId)), UTC_OPT).as("struct")).analyze comparePlans(optimized1, expected1) val query2 = testRelation .select(JsonToStructs(schema, options, - StructsToJson(options, 'struct, UTC_OPT), UTC_OPT).as("struct")) + StructsToJson(options, "struct".attr, UTC_OPT), UTC_OPT).as("struct")) val optimized2 = Optimizer.execute(query2.analyze) - val expected2 = testRelation.select('struct.as("struct")).analyze + val expected2 = testRelation.select("struct".attr.as("struct")).analyze comparePlans(optimized2, expected2) } @@ -157,21 +157,21 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { val options = Map.empty[String, String] val query1 = testRelation2 - .select(GetStructField(JsonToStructs(schema, options, 'json), 0)) + .select(GetStructField(JsonToStructs(schema, options, "json".attr), 0)) val optimized1 = Optimizer.execute(query1.analyze) val prunedSchema1 = StructType.fromDDL("a int") val expected1 = testRelation2 - .select(GetStructField(JsonToStructs(prunedSchema1, options, 'json), 0)).analyze + .select(GetStructField(JsonToStructs(prunedSchema1, options, "json".attr), 0)).analyze comparePlans(optimized1, expected1) val query2 = testRelation2 - .select(GetStructField(JsonToStructs(schema, options, 'json), 1)) + .select(GetStructField(JsonToStructs(schema, options, "json".attr), 1)) val optimized2 = Optimizer.execute(query2.analyze) val prunedSchema2 = StructType.fromDDL("b int") val expected2 = testRelation2 - .select(GetStructField(JsonToStructs(prunedSchema2, options, 'json), 0)).analyze + .select(GetStructField(JsonToStructs(prunedSchema2, options, "json".attr), 0)).analyze comparePlans(optimized2, expected2) } @@ -182,13 +182,13 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { val query1 = testRelation2 .select(GetArrayStructFields( - JsonToStructs(schema1, options, 'json), field1, 0, 2, true).as("a")) + JsonToStructs(schema1, options, "json".attr), field1, 0, 2, true).as("a")) val optimized1 = Optimizer.execute(query1.analyze) val prunedSchema1 = ArrayType(StructType.fromDDL("a int"), containsNull = true) val expected1 = testRelation2 .select(GetArrayStructFields( - JsonToStructs(prunedSchema1, options, 'json), field1, 0, 1, true).as("a")).analyze + JsonToStructs(prunedSchema1, options, "json".attr), field1, 0, 1, true).as("a")).analyze comparePlans(optimized1, expected1) val schema2 = ArrayType( @@ -198,14 +198,14 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { val field2 = schema2.elementType.asInstanceOf[StructType](1) val query2 = testRelation2 .select(GetArrayStructFields( - JsonToStructs(schema2, options, 'json), field2, 1, 2, false).as("b")) + JsonToStructs(schema2, options, "json".attr), field2, 1, 2, false).as("b")) val optimized2 = Optimizer.execute(query2.analyze) val prunedSchema2 = ArrayType( StructType(StructField("b", IntegerType, false) :: Nil), containsNull = false) val expected2 = testRelation2 .select(GetArrayStructFields( - JsonToStructs(prunedSchema2, options, 'json), field2, 0, 1, false).as("b")).analyze + JsonToStructs(prunedSchema2, options, "json".attr), field2, 0, 1, false).as("b")).analyze comparePlans(optimized2, expected2) } @@ -213,7 +213,7 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { val options = Map("mode" -> "failfast") val query1 = testRelation2 - .select(GetStructField(JsonToStructs(schema, options, 'json), 0)) + .select(GetStructField(JsonToStructs(schema, options, "json".attr), 0)) val optimized1 = Optimizer.execute(query1.analyze) comparePlans(optimized1, query1.analyze) @@ -223,7 +223,7 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { val query2 = testRelation2 .select(GetArrayStructFields( - JsonToStructs(schema1, options, 'json), field1, 0, 2, true).as("a")) + JsonToStructs(schema1, options, "json".attr), field1, 0, 2, true).as("a")) val optimized2 = Optimizer.execute(query2.analyze) comparePlans(optimized2, query2.analyze) @@ -237,7 +237,7 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { val nullStruct = namedStruct("a", Literal(null, IntegerType), "b", Literal(null, IntegerType)) val UTC_OPT = Option("UTC") - val json: BoundReference = 'json.string.canBeNull.at(0) + val json: BoundReference = "json".attr.string.canBeNull.at(0) assertEquivalent( testRelation2, @@ -301,7 +301,7 @@ class OptimizeJsonExprsSuite extends PlanTest with ExpressionEvalHelper { val options = Map.empty[String, String] val query = testRelation - .select(JsonToStructs(schema, options, StructsToJson(options, 'struct)).as("struct")) + .select(JsonToStructs(schema, options, StructsToJson(options, "struct".attr)).as("struct")) val optimized = Optimizer.execute(query.analyze) comparePlans(optimized, query.analyze) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeLimitZeroSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeLimitZeroSuite.scala index cf875efc62c9..bce2e368877c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeLimitZeroSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeLimitZeroSuite.scala @@ -36,14 +36,14 @@ class OptimizeLimitZeroSuite extends PlanTest { PropagateEmptyRelation) :: Nil } - val testRelation1 = LocalRelation.fromExternalRows(Seq('a.int), data = Seq(Row(1))) - val testRelation2 = LocalRelation.fromExternalRows(Seq('b.int), data = Seq(Row(1))) + val testRelation1 = LocalRelation.fromExternalRows(Seq("a".attr.int), data = Seq(Row(1))) + val testRelation2 = LocalRelation.fromExternalRows(Seq("b".attr.int), data = Seq(Row(1))) test("Limit 0: return empty local relation") { val query = testRelation1.limit(0) val optimized = Optimize.execute(query.analyze) - val correctAnswer = LocalRelation('a.int) + val correctAnswer = LocalRelation("a".attr.int) comparePlans(optimized, correctAnswer) } @@ -52,7 +52,7 @@ class OptimizeLimitZeroSuite extends PlanTest { val query = LocalLimit(0, testRelation1) val optimized = Optimize.execute(query.analyze) - val correctAnswer = LocalRelation('a.int) + val correctAnswer = LocalRelation("a".attr.int) comparePlans(optimized, correctAnswer) } @@ -61,20 +61,23 @@ class OptimizeLimitZeroSuite extends PlanTest { val query = GlobalLimit(0, testRelation1) val optimized = Optimize.execute(query.analyze) - val correctAnswer = LocalRelation('a.int) + val correctAnswer = LocalRelation("a".attr.int) comparePlans(optimized, correctAnswer) } Seq( - (Inner, LocalRelation('a.int, 'b.int)), - (LeftOuter, Project(Seq('a, Literal(null).cast(IntegerType).as('b)), testRelation1).analyze), - (RightOuter, LocalRelation('a.int, 'b.int)), - (FullOuter, Project(Seq('a, Literal(null).cast(IntegerType).as('b)), testRelation1).analyze) + (Inner, LocalRelation("a".attr.int, "b".attr.int)), + (LeftOuter, Project(Seq("a".attr, Literal(null).cast(IntegerType).as("b")), + testRelation1).analyze), + (RightOuter, LocalRelation("a".attr.int, "b".attr.int)), + (FullOuter, Project(Seq("a".attr, Literal(null).cast(IntegerType).as("b")), + testRelation1).analyze) ).foreach { case (jt, correctAnswer) => test(s"Limit 0: for join type $jt") { val query = testRelation1 - .join(testRelation2.limit(0), joinType = jt, condition = Some('a.attr == 'b.attr)) + .join(testRelation2.limit(0), joinType = jt, + condition = Some("a".attr == "b".attr)) val optimized = Optimize.execute(query.analyze) @@ -83,15 +86,16 @@ class OptimizeLimitZeroSuite extends PlanTest { } test("Limit 0: 3-way join") { - val testRelation3 = LocalRelation.fromExternalRows(Seq('c.int), data = Seq(Row(1))) + val testRelation3 = LocalRelation.fromExternalRows(Seq("c".attr.int), data = Seq(Row(1))) val subJoinQuery = testRelation1 - .join(testRelation2, joinType = Inner, condition = Some('a.attr == 'b.attr)) + .join(testRelation2, joinType = Inner, condition = Some("a".attr == "b".attr)) val query = subJoinQuery - .join(testRelation3.limit(0), joinType = Inner, condition = Some('a.attr == 'c.attr)) + .join(testRelation3.limit(0), joinType = Inner, + condition = Some("a".attr == "c".attr)) val optimized = Optimize.execute(query.analyze) - val correctAnswer = LocalRelation('a.int, 'b.int, 'c.int) + val correctAnswer = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) comparePlans(optimized, correctAnswer) } @@ -101,7 +105,7 @@ class OptimizeLimitZeroSuite extends PlanTest { .intersect(testRelation1.limit(0), isAll = false) val optimized = Optimize.execute(query.analyze) - val correctAnswer = Distinct(LocalRelation('a.int)) + val correctAnswer = Distinct(LocalRelation("a".attr.int)) comparePlans(optimized, correctAnswer) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeWindowFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeWindowFunctionsSuite.scala index cf850bbe21ce..52bfbc15a765 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeWindowFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeWindowFunctionsSuite.scala @@ -31,7 +31,7 @@ class OptimizeWindowFunctionsSuite extends PlanTest { OptimizeWindowFunctions) :: Nil } - val testRelation = LocalRelation('a.double, 'b.double, 'c.string) + val testRelation = LocalRelation("a".attr.double, "b".attr.double, "c".attr.string) val a = testRelation.output(0) val b = testRelation.output(1) val c = testRelation.output(2) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeWithFieldsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeWithFieldsSuite.scala index b093b39cc4b8..97b08e225f12 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeWithFieldsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeWithFieldsSuite.scala @@ -32,21 +32,22 @@ class OptimizeWithFieldsSuite extends PlanTest { OptimizeUpdateFields, SimplifyExtractValueOps) :: Nil } - private val testRelation = LocalRelation('a.struct('a1.int)) - private val testRelation2 = LocalRelation('a.struct('a1.int).notNull) + private val testRelation = LocalRelation("a".attr.struct("a1".attr.int)) + private val testRelation2 = LocalRelation("a".attr.struct("a1".attr.int).notNull) test("combines two adjacent UpdateFields Expressions") { val originalQuery = testRelation .select(Alias( UpdateFields( UpdateFields( - 'a, + "a".attr, WithField("b1", Literal(4)) :: Nil), WithField("c1", Literal(5)) :: Nil), "out")()) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select(Alias(UpdateFields('a, WithField("b1", Literal(4)) :: WithField("c1", Literal(5)) :: + .select(Alias( + UpdateFields("a".attr, WithField("b1", Literal(4)) :: WithField("c1", Literal(5)) :: Nil), "out")()) .analyze @@ -59,14 +60,15 @@ class OptimizeWithFieldsSuite extends PlanTest { UpdateFields( UpdateFields( UpdateFields( - 'a, + "a".attr, WithField("b1", Literal(4)) :: Nil), WithField("c1", Literal(5)) :: Nil), WithField("d1", Literal(6)) :: Nil), "out")()) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select(Alias(UpdateFields('a, WithField("b1", Literal(4)) :: WithField("c1", Literal(5)) :: + .select(Alias( + UpdateFields("a".attr, WithField("b1", Literal(4)) :: WithField("c1", Literal(5)) :: WithField("d1", Literal(6)) :: Nil), "out")()) .analyze @@ -76,7 +78,7 @@ class OptimizeWithFieldsSuite extends PlanTest { test("SPARK-32941: optimize WithFields followed by GetStructField") { val originalQuery = testRelation2 .select(Alias( - GetStructField(UpdateFields('a, + GetStructField(UpdateFields("a".attr, WithField("b1", Literal(4)) :: Nil), 1), "out")()) val optimized = Optimize.execute(originalQuery.analyze) @@ -90,16 +92,16 @@ class OptimizeWithFieldsSuite extends PlanTest { test("SPARK-32941: optimize WithFields chain - case insensitive") { val originalQuery = testRelation .select( - Alias(UpdateFields('a, + Alias(UpdateFields("a".attr, WithField("b1", Literal(4)) :: WithField("b1", Literal(5)) :: Nil), "out1")(), - Alias(UpdateFields('a, + Alias(UpdateFields("a".attr, WithField("b1", Literal(4)) :: WithField("B1", Literal(5)) :: Nil), "out2")()) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select( - Alias(UpdateFields('a, WithField("b1", Literal(5)) :: Nil), "out1")(), - Alias(UpdateFields('a, WithField("B1", Literal(5)) :: Nil), "out2")()) + Alias(UpdateFields("a".attr, WithField("b1", Literal(5)) :: Nil), "out1")(), + Alias(UpdateFields("a".attr, WithField("B1", Literal(5)) :: Nil), "out2")()) .analyze comparePlans(optimized, correctAnswer) @@ -109,17 +111,17 @@ class OptimizeWithFieldsSuite extends PlanTest { withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { val originalQuery = testRelation .select( - Alias(UpdateFields('a, + Alias(UpdateFields("a".attr, WithField("b1", Literal(4)) :: WithField("b1", Literal(5)) :: Nil), "out1")(), - Alias(UpdateFields('a, + Alias(UpdateFields("a".attr, WithField("b1", Literal(4)) :: WithField("B1", Literal(5)) :: Nil), "out2")()) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select( - Alias(UpdateFields('a, WithField("b1", Literal(5)) :: Nil), "out1")(), + Alias(UpdateFields("a".attr, WithField("b1", Literal(5)) :: Nil), "out1")(), Alias( - UpdateFields('a, + UpdateFields("a".attr, WithField("b1", Literal(4)) :: WithField("B1", Literal(5)) :: Nil), "out2")()) .analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala index 1187950c0424..b40f0d7a763f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala @@ -39,9 +39,9 @@ class OptimizerLoggingSuite extends PlanTest { private def verifyLog(expectedLevel: Level, expectedRulesOrBatches: Seq[String]): Unit = { val logAppender = new LogAppender("optimizer rules") withLogAppender(logAppender, level = Some(Level.TRACE)) { - val input = LocalRelation('a.int, 'b.string, 'c.double) - val query = input.select('a, 'b).select('a).where('a > 1).analyze - val expected = input.where('a > 1).select('a).analyze + val input = LocalRelation("a".attr.int, "b".attr.string, "c".attr.double) + val query = input.select("a".attr, "b".attr).select("a".attr).where("a".attr > 1).analyze + val expected = input.where("a".attr > 1).select("a".attr).analyze comparePlans(Optimize.execute(query), expected) } val events = logAppender.loggingEvents.filter { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerRuleExclusionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerRuleExclusionSuite.scala index a277a2d339e9..8cfb4edfd94a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerRuleExclusionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerRuleExclusionSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.internal.SQLConf.OPTIMIZER_EXCLUDED_RULES class OptimizerRuleExclusionSuite extends PlanTest { - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) private def verifyExcludedRules(optimizer: Optimizer, rulesToExclude: Seq[String]): Unit = { val nonExcludableRules = optimizer.nonExcludableRules @@ -121,9 +121,9 @@ class OptimizerRuleExclusionSuite extends PlanTest { PropagateEmptyRelation.ruleName, CombineUnions.ruleName) - val testRelation1 = LocalRelation('a.int, 'b.int, 'c.int) - val testRelation2 = LocalRelation('a.int, 'b.int, 'c.int) - val testRelation3 = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation1 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) + val testRelation2 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) + val testRelation3 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) withSQLConf( OPTIMIZER_EXCLUDED_RULES.key -> excludedRules.foldLeft("")((l, r) => l + "," + r)) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerStructuralIntegrityCheckerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerStructuralIntegrityCheckerSuite.scala index f4a52180373c..bb1c56d11d97 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerStructuralIntegrityCheckerSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerStructuralIntegrityCheckerSuite.scala @@ -62,8 +62,8 @@ class OptimizerStructuralIntegrityCheckerSuite extends PlanTest { test("check for invalid plan after execution of rule - special expression in wrong operator") { val analyzed = - Aggregate(Nil, Seq[NamedExpression](max('id) as 'm), - LocalRelation('id.long)).analyze + Aggregate(Nil, Seq[NamedExpression](max("id".attr) as "m"), + LocalRelation("id".attr.long)).analyze assert(analyzed.resolved) // Should fail verification with the OptimizeRuleBreakSI rule @@ -80,8 +80,8 @@ class OptimizerStructuralIntegrityCheckerSuite extends PlanTest { test("check for invalid plan before execution of any rule") { val analyzed = - Aggregate(Nil, Seq[NamedExpression](max('id) as 'm), - LocalRelation('id.long)).analyze + Aggregate(Nil, Seq[NamedExpression](max("id".attr) as "m"), + LocalRelation("id".attr.long)).analyze val invalidPlan = OptimizeRuleBreakSI.apply(analyzed) // Should fail verification right at the beginning diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala index 893c111c2906..ed64d07b0c5c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala @@ -36,20 +36,20 @@ class OuterJoinEliminationSuite extends PlanTest { PushPredicateThroughJoin) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) - val testRelation1 = LocalRelation('d.int, 'e.int, 'f.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) + val testRelation1 = LocalRelation("d".attr.int, "e".attr.int, "f".attr.int) test("joins: full outer to inner") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") val originalQuery = x.join(y, FullOuter, Option("x.a".attr === "y.d".attr)) .where("x.b".attr >= 1 && "y.d".attr >= 2) val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('b >= 1) - val right = testRelation1.where('d >= 2) + val left = testRelation.where("b".attr >= 1) + val right = testRelation1.where("d".attr >= 2) val correctAnswer = left.join(right, Inner, Option("a".attr === "d".attr)).analyze @@ -57,15 +57,15 @@ class OuterJoinEliminationSuite extends PlanTest { } test("joins: full outer to right") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") val originalQuery = x.join(y, FullOuter, Option("x.a".attr === "y.d".attr)).where("y.d".attr > 2) val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation - val right = testRelation1.where('d > 2) + val right = testRelation1.where("d".attr > 2) val correctAnswer = left.join(right, RightOuter, Option("a".attr === "d".attr)).analyze @@ -73,14 +73,14 @@ class OuterJoinEliminationSuite extends PlanTest { } test("joins: full outer to left") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") val originalQuery = x.join(y, FullOuter, Option("x.a".attr === "y.d".attr)).where("x.a".attr <=> 2) val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('a <=> 2) + val left = testRelation.where("a".attr <=> 2) val right = testRelation1 val correctAnswer = left.join(right, LeftOuter, Option("a".attr === "d".attr)).analyze @@ -89,14 +89,14 @@ class OuterJoinEliminationSuite extends PlanTest { } test("joins: right to inner") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") val originalQuery = x.join(y, RightOuter, Option("x.a".attr === "y.d".attr)).where("x.b".attr > 2) val optimized = Optimize.execute(originalQuery.analyze) - val left = testRelation.where('b > 2) + val left = testRelation.where("b".attr > 2) val right = testRelation1 val correctAnswer = left.join(right, Inner, Option("a".attr === "d".attr)).analyze @@ -105,8 +105,8 @@ class OuterJoinEliminationSuite extends PlanTest { } test("joins: left to inner") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") val originalQuery = x.join(y, LeftOuter, Option("x.a".attr === "y.d".attr)) @@ -114,7 +114,7 @@ class OuterJoinEliminationSuite extends PlanTest { val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation - val right = testRelation1.where('e.isNotNull) + val right = testRelation1.where("e".attr.isNotNull) val correctAnswer = left.join(right, Inner, Option("a".attr === "d".attr)).analyze @@ -123,34 +123,34 @@ class OuterJoinEliminationSuite extends PlanTest { // evaluating if mixed OR and NOT expressions can eliminate all null-supplying rows test("joins: left to inner with complicated filter predicates #1") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") val originalQuery = x.join(y, LeftOuter, Option("x.a".attr === "y.d".attr)) - .where(!'e.isNull || ('d.isNotNull && 'f.isNull)) + .where(!"e".attr.isNull || ("d".attr.isNotNull && "f".attr.isNull)) val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation - val right = testRelation1.where(!'e.isNull || ('d.isNotNull && 'f.isNull)) + val right = testRelation1.where(!"e".attr.isNull || ("d".attr.isNotNull && "f".attr.isNull)) val correctAnswer = left.join(right, Inner, Option("a".attr === "d".attr)).analyze comparePlans(optimized, correctAnswer) } - // eval(emptyRow) of 'e.in(1, 2) will return null instead of false + // eval(emptyRow) of "e".attr.in(1, 2) will return null instead of false test("joins: left to inner with complicated filter predicates #2") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") val originalQuery = x.join(y, LeftOuter, Option("x.a".attr === "y.d".attr)) - .where('e.in(1, 2)) + .where("e".attr.in(1, 2)) val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation - val right = testRelation1.where('e.in(1, 2)) + val right = testRelation1.where("e".attr.in(1, 2)) val correctAnswer = left.join(right, Inner, Option("a".attr === "d".attr)).analyze @@ -159,16 +159,17 @@ class OuterJoinEliminationSuite extends PlanTest { // evaluating if mixed OR and AND expressions can eliminate all null-supplying rows test("joins: left to inner with complicated filter predicates #3") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") val originalQuery = x.join(y, LeftOuter, Option("x.a".attr === "y.d".attr)) - .where((!'e.isNull || ('d.isNotNull && 'f.isNull)) && 'e.isNull) + .where((!"e".attr.isNull || ("d".attr.isNotNull && "f".attr.isNull)) && "e".attr.isNull) val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation - val right = testRelation1.where((!'e.isNull || ('d.isNotNull && 'f.isNull)) && 'e.isNull) + val right = testRelation1.where((!"e".attr.isNull || ("d".attr.isNotNull && + "f".attr.isNull)) && "e".attr.isNull) val correctAnswer = left.join(right, Inner, Option("a".attr === "d".attr)).analyze @@ -179,8 +180,8 @@ class OuterJoinEliminationSuite extends PlanTest { // can eliminate all null-supplying rows // FULL OUTER => INNER test("joins: left to inner with complicated filter predicates #4") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") val originalQuery = x.join(y, FullOuter, Option("x.a".attr === "y.d".attr)) @@ -196,8 +197,8 @@ class OuterJoinEliminationSuite extends PlanTest { } test("joins: no outer join elimination if the filter is not NULL eliminated") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") val originalQuery = x.join(y, FullOuter, Option("x.a".attr === "y.d".attr)) @@ -215,8 +216,8 @@ class OuterJoinEliminationSuite extends PlanTest { } test("joins: no outer join elimination if the filter's constraints are not NULL eliminated") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") val originalQuery = x.join(y, FullOuter, Option("x.a".attr === "y.d".attr)) @@ -235,8 +236,8 @@ class OuterJoinEliminationSuite extends PlanTest { test("no outer join elimination if constraint propagation is disabled") { withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "false") { - val x = testRelation.subquery('x) - val y = testRelation1.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation1.subquery("y") // The predicate "x.b + y.d >= 3" will be inferred constraints like: // "x.b != null" and "y.d != null", if constraint propagation is enabled. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala index 5c980abdd8f5..1028b9c7c54a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala @@ -53,11 +53,11 @@ class PropagateEmptyRelationSuite extends PlanTest { CollapseProject) :: Nil } - val testRelation1 = LocalRelation.fromExternalRows(Seq('a.int), data = Seq(Row(1))) - val testRelation2 = LocalRelation.fromExternalRows(Seq('b.int), data = Seq(Row(1))) + val testRelation1 = LocalRelation.fromExternalRows(Seq("a".attr.int), data = Seq(Row(1))) + val testRelation2 = LocalRelation.fromExternalRows(Seq("b".attr.int), data = Seq(Row(1))) val metadata = new MetadataBuilder().putLong("test", 1).build() - val testRelation3 = - LocalRelation.fromExternalRows(Seq('c.int.notNull.withMetadata(metadata)), data = Seq(Row(1))) + val testRelation3 = LocalRelation.fromExternalRows( + Seq("c".attr.int.notNull.withMetadata(metadata)), data = Seq(Row(1))) test("propagate empty relation through Union") { val query = testRelation1 @@ -65,7 +65,7 @@ class PropagateEmptyRelationSuite extends PlanTest { .union(testRelation2.where(false)) val optimized = Optimize.execute(query.analyze) - val correctAnswer = LocalRelation('a.int) + val correctAnswer = LocalRelation("a".attr.int) comparePlans(optimized, correctAnswer) } @@ -78,7 +78,7 @@ class PropagateEmptyRelationSuite extends PlanTest { val query2 = testRelation1.where(false).union(testRelation2) val optimized2 = Optimize.execute(query2.analyze) - val correctAnswer2 = testRelation2.select('b.as('a)).analyze + val correctAnswer2 = testRelation2.select("b".attr.as("a")).analyze comparePlans(optimized2, correctAnswer2) val query3 = testRelation1.union(testRelation2.where(false)).union(testRelation3) @@ -88,7 +88,7 @@ class PropagateEmptyRelationSuite extends PlanTest { val query4 = testRelation1.where(false).union(testRelation2).union(testRelation3) val optimized4 = Optimize.execute(query4.analyze) - val correctAnswer4 = testRelation2.union(testRelation3).select('b.as('a)).analyze + val correctAnswer4 = testRelation2.union(testRelation3).select("b".attr.as("a")).analyze comparePlans(optimized4, correctAnswer4) // Nullability can change from nullable to non-nullable @@ -115,39 +115,42 @@ class PropagateEmptyRelationSuite extends PlanTest { (true, true, LeftAnti, None), (true, true, LeftSemi, None), - (true, false, Inner, Some(LocalRelation('a.int, 'b.int))), - (true, false, Cross, Some(LocalRelation('a.int, 'b.int))), - (true, false, LeftOuter, - Some(Project(Seq('a, Literal(null).cast(IntegerType).as('b)), testRelation1).analyze)), - (true, false, RightOuter, Some(LocalRelation('a.int, 'b.int))), - (true, false, FullOuter, - Some(Project(Seq('a, Literal(null).cast(IntegerType).as('b)), testRelation1).analyze)), + (true, false, Inner, Some(LocalRelation("a".attr.int, "b".attr.int))), + (true, false, Cross, Some(LocalRelation("a".attr.int, "b".attr.int))), + (true, false, LeftOuter, Some(Project(Seq("a".attr, + Literal(null).cast(IntegerType).as("b")), testRelation1).analyze)), + (true, false, RightOuter, Some(LocalRelation("a".attr.int, "b".attr.int))), + (true, false, FullOuter, Some(Project(Seq("a".attr, + Literal(null).cast(IntegerType).as("b")), testRelation1).analyze)), (true, false, LeftAnti, Some(testRelation1)), - (true, false, LeftSemi, Some(LocalRelation('a.int))), + (true, false, LeftSemi, Some(LocalRelation("a".attr.int))), - (false, true, Inner, Some(LocalRelation('a.int, 'b.int))), - (false, true, Cross, Some(LocalRelation('a.int, 'b.int))), - (false, true, LeftOuter, Some(LocalRelation('a.int, 'b.int))), + (false, true, Inner, Some(LocalRelation("a".attr.int, "b".attr.int))), + (false, true, Cross, Some(LocalRelation("a".attr.int, "b".attr.int))), + (false, true, LeftOuter, Some(LocalRelation("a".attr.int, "b".attr.int))), (false, true, RightOuter, - Some(Project(Seq(Literal(null).cast(IntegerType).as('a), 'b), testRelation2).analyze)), + Some(Project(Seq(Literal(null).cast(IntegerType).as("a"), "b".attr), + testRelation2).analyze)), (false, true, FullOuter, - Some(Project(Seq(Literal(null).cast(IntegerType).as('a), 'b), testRelation2).analyze)), - (false, true, LeftAnti, Some(LocalRelation('a.int))), - (false, true, LeftSemi, Some(LocalRelation('a.int))), - - (false, false, Inner, Some(LocalRelation('a.int, 'b.int))), - (false, false, Cross, Some(LocalRelation('a.int, 'b.int))), - (false, false, LeftOuter, Some(LocalRelation('a.int, 'b.int))), - (false, false, RightOuter, Some(LocalRelation('a.int, 'b.int))), - (false, false, FullOuter, Some(LocalRelation('a.int, 'b.int))), - (false, false, LeftAnti, Some(LocalRelation('a.int))), - (false, false, LeftSemi, Some(LocalRelation('a.int))) + Some(Project(Seq(Literal(null).cast(IntegerType).as("a"), "b".attr), + testRelation2).analyze)), + (false, true, LeftAnti, Some(LocalRelation("a".attr.int))), + (false, true, LeftSemi, Some(LocalRelation("a".attr.int))), + + (false, false, Inner, Some(LocalRelation("a".attr.int, "b".attr.int))), + (false, false, Cross, Some(LocalRelation("a".attr.int, "b".attr.int))), + (false, false, LeftOuter, Some(LocalRelation("a".attr.int, "b".attr.int))), + (false, false, RightOuter, Some(LocalRelation("a".attr.int, "b".attr.int))), + (false, false, FullOuter, Some(LocalRelation("a".attr.int, "b".attr.int))), + (false, false, LeftAnti, Some(LocalRelation("a".attr.int))), + (false, false, LeftSemi, Some(LocalRelation("a".attr.int))) ) testcases.foreach { case (left, right, jt, answer) => val query = testRelation1 .where(left) - .join(testRelation2.where(right), joinType = jt, condition = Some('a.attr == 'b.attr)) + .join(testRelation2.where(right), joinType = jt, + condition = Some("a".attr == "b".attr)) val optimized = Optimize.execute(query.analyze) val correctAnswer = answer.getOrElse(OptimizeWithoutPropagateEmptyRelation.execute(query.analyze)) @@ -158,19 +161,19 @@ class PropagateEmptyRelationSuite extends PlanTest { test("propagate empty relation through UnaryNode") { val query = testRelation1 .where(false) - .select('a) - .groupBy('a)('a) - .where('a > 1) - .orderBy('a.asc) + .select("a".attr) + .groupBy("a".attr)("a".attr) + .where("a".attr > 1) + .orderBy("a".attr.asc) val optimized = Optimize.execute(query.analyze) - val correctAnswer = LocalRelation('a.int) + val correctAnswer = LocalRelation("a".attr.int) comparePlans(optimized, correctAnswer) } test("propagate empty streaming relation through multiple UnaryNode") { - val output = Seq('a.int) + val output = Seq("a".attr.int) val data = Seq(Row(1)) val schema = StructType.fromAttributes(output) val converter = CatalystTypeConverters.createToCatalystConverter(schema) @@ -181,10 +184,10 @@ class PropagateEmptyRelationSuite extends PlanTest { val query = relation .where(false) - .select('a) - .where('a > 1) - .where('a =!= 200) - .orderBy('a.asc) + .select("a".attr) + .where("a".attr > 1) + .where("a".attr =!= 200) + .orderBy("a".attr.asc) val optimized = Optimize.execute(query.analyze) val correctAnswer = LocalRelation(output, isStreaming = true) @@ -193,7 +196,7 @@ class PropagateEmptyRelationSuite extends PlanTest { } test("don't propagate empty streaming relation through agg") { - val output = Seq('a.int) + val output = Seq("a".attr.int) val data = Seq(Row(1)) val schema = StructType.fromAttributes(output) val converter = CatalystTypeConverters.createToCatalystConverter(schema) @@ -203,7 +206,7 @@ class PropagateEmptyRelationSuite extends PlanTest { isStreaming = true) val query = relation - .groupBy('a)('a) + .groupBy("a".attr)("a".attr) val optimized = Optimize.execute(query.analyze) val correctAnswer = query.analyze @@ -214,17 +217,17 @@ class PropagateEmptyRelationSuite extends PlanTest { test("don't propagate non-empty local relation") { val query = testRelation1 .where(true) - .groupBy('a)('a) - .where('a > 1) - .orderBy('a.asc) - .select('a) + .groupBy("a".attr)("a".attr) + .where("a".attr > 1) + .orderBy("a".attr.asc) + .select("a".attr) val optimized = Optimize.execute(query.analyze) val correctAnswer = testRelation1 - .where('a > 1) - .groupBy('a)('a) - .orderBy('a.asc) - .select('a) + .where("a".attr > 1) + .groupBy("a".attr)("a".attr) + .orderBy("a".attr.asc) + .select("a".attr) comparePlans(optimized, correctAnswer.analyze) } @@ -232,10 +235,10 @@ class PropagateEmptyRelationSuite extends PlanTest { test("propagate empty relation through Aggregate with grouping expressions") { val query = testRelation1 .where(false) - .groupBy('a)('a, ('a + 1).as('x)) + .groupBy("a".attr)("a".attr, ("a".attr + 1).as("x")) val optimized = Optimize.execute(query.analyze) - val correctAnswer = LocalRelation('a.int, 'x.int).analyze + val correctAnswer = LocalRelation("a".attr.int, "x".attr.int).analyze comparePlans(optimized, correctAnswer) } @@ -246,14 +249,14 @@ class PropagateEmptyRelationSuite extends PlanTest { .groupBy()() val optimized = Optimize.execute(query.analyze) - val correctAnswer = LocalRelation('a.int).groupBy()().analyze + val correctAnswer = LocalRelation("a".attr.int).groupBy()().analyze comparePlans(optimized, correctAnswer) } test("propagate empty relation keeps the plan resolved") { val query = testRelation1.join( - LocalRelation('a.int, 'b.int), UsingJoin(FullOuter, "a" :: Nil), None) + LocalRelation("a".attr.int, "b".attr.int), UsingJoin(FullOuter, "a" :: Nil), None) val optimized = Optimize.execute(query.analyze) assert(optimized.resolved) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PruneFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PruneFiltersSuite.scala index 15fbe3c5b0a1..108a9ad584bf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PruneFiltersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PruneFiltersSuite.scala @@ -39,11 +39,11 @@ class PruneFiltersSuite extends PlanTest { PushPredicateThroughJoin) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) test("Constraints of isNull + LeftOuter") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val query = x.where("x.b".attr.isNull).join(y, LeftOuter) val queryWithUselessFilter = query.where("x.b".attr.isNull) @@ -55,15 +55,15 @@ class PruneFiltersSuite extends PlanTest { } test("Constraints of unionall") { - val tr1 = LocalRelation('a.int, 'b.int, 'c.int) - val tr2 = LocalRelation('d.int, 'e.int, 'f.int) - val tr3 = LocalRelation('g.int, 'h.int, 'i.int) + val tr1 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) + val tr2 = LocalRelation("d".attr.int, "e".attr.int, "f".attr.int) + val tr3 = LocalRelation("g".attr.int, "h".attr.int, "i".attr.int) val query = - tr1.where('a.attr > 10) - .union(tr2.where('d.attr > 10) - .union(tr3.where('g.attr > 10))) - val queryWithUselessFilter = query.where('a.attr > 10) + tr1.where("a".attr > 10) + .union(tr2.where("d".attr > 10) + .union(tr3.where("g".attr > 10))) + val queryWithUselessFilter = query.where("a".attr > 10) val optimized = Optimize.execute(queryWithUselessFilter.analyze) val correctAnswer = query.analyze @@ -72,17 +72,17 @@ class PruneFiltersSuite extends PlanTest { } test("Pruning multiple constraints in the same run") { - val tr1 = LocalRelation('a.int, 'b.int, 'c.int).subquery('tr1) - val tr2 = LocalRelation('a.int, 'd.int, 'e.int).subquery('tr2) + val tr1 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int).subquery("tr1") + val tr2 = LocalRelation("a".attr.int, "d".attr.int, "e".attr.int).subquery("tr2") val query = tr1 .where("tr1.a".attr > 10 || "tr1.c".attr < 10) - .join(tr2.where('d.attr < 100), Inner, Some("tr1.a".attr === "tr2.a".attr)) + .join(tr2.where("d".attr < 100), Inner, Some("tr1.a".attr === "tr2.a".attr)) // different order of "tr2.a" and "tr1.a" val queryWithUselessFilter = query.where( ("tr1.a".attr > 10 || "tr1.c".attr < 10) && - 'd.attr < 100 && + "d".attr < 100 && "tr2.a".attr === "tr1.a".attr) val optimized = Optimize.execute(queryWithUselessFilter.analyze) @@ -92,21 +92,21 @@ class PruneFiltersSuite extends PlanTest { } test("Partial pruning") { - val tr1 = LocalRelation('a.int, 'b.int, 'c.int).subquery('tr1) - val tr2 = LocalRelation('a.int, 'd.int, 'e.int).subquery('tr2) + val tr1 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int).subquery("tr1") + val tr2 = LocalRelation("a".attr.int, "d".attr.int, "e".attr.int).subquery("tr2") // One of the filter condition does not exist in the constraints of its child // Thus, the filter is not removed val query = tr1 .where("tr1.a".attr > 10) - .join(tr2.where('d.attr < 100), Inner, Some("tr1.a".attr === "tr2.d".attr)) + .join(tr2.where("d".attr < 100), Inner, Some("tr1.a".attr === "tr2.d".attr)) val queryWithExtraFilters = - query.where("tr1.a".attr > 10 && 'd.attr < 100 && "tr1.a".attr === "tr2.a".attr) + query.where("tr1.a".attr > 10 && "d".attr < 100 && "tr1.a".attr === "tr2.a".attr) val optimized = Optimize.execute(queryWithExtraFilters.analyze) val correctAnswer = tr1 .where("tr1.a".attr > 10) - .join(tr2.where('d.attr < 100), + .join(tr2.where("d".attr < 100), Inner, Some("tr1.a".attr === "tr2.a".attr && "tr1.a".attr === "tr2.d".attr)).analyze @@ -114,8 +114,8 @@ class PruneFiltersSuite extends PlanTest { } test("No predicate is pruned") { - val x = testRelation.subquery('x) - val y = testRelation.subquery('y) + val x = testRelation.subquery("x") + val y = testRelation.subquery("y") val query = x.where("x.b".attr.isNull).join(y, LeftOuter) val queryWithExtraFilters = query.where("x.b".attr.isNotNull) @@ -129,24 +129,26 @@ class PruneFiltersSuite extends PlanTest { } test("Nondeterministic predicate is not pruned") { - val originalQuery = testRelation.where(Rand(10) > 5).select('a).where(Rand(10) > 5).analyze + val originalQuery = + testRelation.where(Rand(10) > 5).select("a".attr).where(Rand(10) > 5).analyze val optimized = Optimize.execute(originalQuery) - val correctAnswer = testRelation.where(Rand(10) > 5).where(Rand(10) > 5).select('a).analyze + val correctAnswer = + testRelation.where(Rand(10) > 5).where(Rand(10) > 5).select("a".attr).analyze comparePlans(optimized, correctAnswer) } test("No pruning when constraint propagation is disabled") { - val tr1 = LocalRelation('a.int, 'b.int, 'c.int).subquery('tr1) - val tr2 = LocalRelation('a.int, 'd.int, 'e.int).subquery('tr2) + val tr1 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int).subquery("tr1") + val tr2 = LocalRelation("a".attr.int, "d".attr.int, "e".attr.int).subquery("tr2") val query = tr1 .where("tr1.a".attr > 10 || "tr1.c".attr < 10) - .join(tr2.where('d.attr < 100), Inner, Some("tr1.a".attr === "tr2.a".attr)) + .join(tr2.where("d".attr < 100), Inner, Some("tr1.a".attr === "tr2.a".attr)) val queryWithUselessFilter = query.where( ("tr1.a".attr > 10 || "tr1.c".attr < 10) && - 'd.attr < 100) + "d".attr < 100) withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "false") { val optimized = Optimize.execute(queryWithUselessFilter.analyze) @@ -155,7 +157,7 @@ class PruneFiltersSuite extends PlanTest { // and duplicate filters. val correctAnswer = tr1 .where("tr1.a".attr > 10 || "tr1.c".attr < 10).where("tr1.a".attr > 10 || "tr1.c".attr < 10) - .join(tr2.where('d.attr < 100).where('d.attr < 100), + .join(tr2.where("d".attr < 100).where("d".attr < 100), Inner, Some("tr1.a".attr === "tr2.a".attr)).analyze comparePlans(optimized, correctAnswer) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullupCorrelatedPredicatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullupCorrelatedPredicatesSuite.scala index ae9a694b5044..a4c1f20b19e0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullupCorrelatedPredicatesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullupCorrelatedPredicatesSuite.scala @@ -34,18 +34,18 @@ class PullupCorrelatedPredicatesSuite extends PlanTest { PullupCorrelatedPredicates) :: Nil } - val testRelation = LocalRelation('a.int, 'b.double) - val testRelation2 = LocalRelation('c.int, 'd.double) + val testRelation = LocalRelation("a".attr.int, "b".attr.double) + val testRelation2 = LocalRelation("c".attr.int, "d".attr.double) test("PullupCorrelatedPredicates should not produce unresolved plan") { val subPlan = testRelation2 - .where('b < 'd) - .select('c) + .where("b".attr < "d".attr) + .select("c".attr) val inSubquery = testRelation - .where(InSubquery(Seq('a), ListQuery(subPlan))) - .select('a).analyze + .where(InSubquery(Seq("a".attr), ListQuery(subPlan))) + .select("a".attr).analyze assert(inSubquery.resolved) val optimized = Optimize.execute(inSubquery) @@ -55,12 +55,12 @@ class PullupCorrelatedPredicatesSuite extends PlanTest { test("PullupCorrelatedPredicates in correlated subquery idempotency check") { val subPlan = testRelation2 - .where('b < 'd) - .select('c) + .where("b".attr < "d".attr) + .select("c".attr) val inSubquery = testRelation - .where(InSubquery(Seq('a), ListQuery(subPlan))) - .select('a).analyze + .where(InSubquery(Seq("a".attr), ListQuery(subPlan))) + .select("a".attr).analyze assert(inSubquery.resolved) val optimized = Optimize.execute(inSubquery) @@ -76,7 +76,7 @@ class PullupCorrelatedPredicatesSuite extends PlanTest { val existsSubquery = testRelation .where(Exists(subPlan)) - .select('a).analyze + .select("a".attr).analyze assert(existsSubquery.resolved) val optimized = Optimize.execute(existsSubquery) @@ -88,11 +88,11 @@ class PullupCorrelatedPredicatesSuite extends PlanTest { val subPlan = testRelation2 .where('b === 'd && 'd === 1) - .select(max('d)) + .select(max("d".attr)) val scalarSubquery = testRelation .where(ScalarSubquery(subPlan) === 1) - .select('a).analyze + .select("a".attr).analyze val optimized = Optimize.execute(scalarSubquery) val doubleOptimized = Optimize.execute(optimized) @@ -100,8 +100,8 @@ class PullupCorrelatedPredicatesSuite extends PlanTest { } test("PullupCorrelatedPredicates should handle deletes") { - val subPlan = testRelation2.where('a === 'c).select('c) - val cond = InSubquery(Seq('a), ListQuery(subPlan)) + val subPlan = testRelation2.where('a === "c".attr).select("c".attr) + val cond = InSubquery(Seq("a".attr), ListQuery(subPlan)) val deletePlan = DeleteFromTable(testRelation, Some(cond)).analyze assert(deletePlan.resolved) @@ -118,8 +118,8 @@ class PullupCorrelatedPredicatesSuite extends PlanTest { } test("PullupCorrelatedPredicates should handle updates") { - val subPlan = testRelation2.where('a === 'c).select('c) - val cond = InSubquery(Seq('a), ListQuery(subPlan)) + val subPlan = testRelation2.where('a === "c".attr).select("c".attr) + val cond = InSubquery(Seq("a".attr), ListQuery(subPlan)) val updatePlan = UpdateTable(testRelation, Seq.empty, Some(cond)).analyze assert(updatePlan.resolved) @@ -136,16 +136,16 @@ class PullupCorrelatedPredicatesSuite extends PlanTest { } test("PullupCorrelatedPredicates should handle merge") { - val testRelation3 = LocalRelation('e.int, 'f.double) - val subPlan = testRelation3.where('a === 'e).select('e) - val cond = InSubquery(Seq('a), ListQuery(subPlan)) + val testRelation3 = LocalRelation("e".attr.int, "f".attr.double) + val subPlan = testRelation3.where('a === "e".attr).select("e".attr) + val cond = InSubquery(Seq("a".attr), ListQuery(subPlan)) val mergePlan = MergeIntoTable( testRelation, testRelation2, cond, Seq(DeleteAction(None)), - Seq(InsertAction(None, Seq(Assignment('a, 'c), Assignment('b, 'd))))) + Seq(InsertAction(None, Seq(Assignment("a".attr, "c".attr), Assignment("b".attr, "d".attr))))) val analyzedMergePlan = mergePlan.analyze assert(analyzedMergePlan.resolved) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushFoldableIntoBranchesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushFoldableIntoBranchesSuite.scala index cb90a398604f..86dcd8d69c1b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushFoldableIntoBranchesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushFoldableIntoBranchesSuite.scala @@ -38,7 +38,7 @@ class PushFoldableIntoBranchesSuite BooleanSimplification, ConstantFolding, SimplifyConditionals, PushFoldableIntoBranches) :: Nil } - private val relation = LocalRelation('a.int, 'b.int, 'c.boolean) + private val relation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.boolean) private val a = EqualTo(UnresolvedAttribute("a"), Literal(100)) private val b = UnresolvedAttribute("b") private val c = EqualTo(UnresolvedAttribute("c"), Literal(true)) @@ -271,10 +271,10 @@ class PushFoldableIntoBranchesSuite test("SPARK-33884: simplify CaseWhen clauses with (true and false) and (false and true)") { assertEquivalent( - EqualTo(CaseWhen(Seq(('a > 10, Literal(0))), Literal(1)), Literal(0)), - 'a > 10 <=> TrueLiteral) + EqualTo(CaseWhen(Seq(("a".attr > 10, Literal(0))), Literal(1)), Literal(0)), + "a".attr > 10 <=> TrueLiteral) assertEquivalent( - EqualTo(CaseWhen(Seq(('a > 10, Literal(0))), Literal(1)), Literal(1)), - Not('a > 10 <=> TrueLiteral)) + EqualTo(CaseWhen(Seq(("a".attr > 10, Literal(0))), Literal(1)), Literal(1)), + Not("a".attr > 10 <=> TrueLiteral)) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushProjectThroughUnionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushProjectThroughUnionSuite.scala index 294d29842b04..934e8ec6d904 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushProjectThroughUnionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushProjectThroughUnionSuite.scala @@ -33,19 +33,19 @@ class PushProjectThroughUnionSuite extends PlanTest { test("SPARK-25450 PushProjectThroughUnion rule uses the same exprId for project expressions " + "in each Union child, causing mistakes in constant propagation") { - val testRelation1 = LocalRelation('a.string, 'b.int, 'c.string) - val testRelation2 = LocalRelation('d.string, 'e.int, 'f.string) + val testRelation1 = LocalRelation("a".attr.string, "b".attr.int, "c".attr.string) + val testRelation2 = LocalRelation("d".attr.string, "e".attr.int, "f".attr.string) val query = testRelation1 - .union(testRelation2.select("bar".as("d"), 'e, 'f)) - .select('a.as("n")) - .select('n, "dummy").analyze + .union(testRelation2.select("bar".as("d"), "e".attr, "f".attr)) + .select("a".attr.as("n")) + .select("n".attr, "dummy").analyze val optimized = Optimize.execute(query) val expected = testRelation1 - .select('a.as("n")) - .select('n, "dummy") + .select("a".attr.as("n")) + .select("n".attr, "dummy") .union(testRelation2 - .select("bar".as("d"), 'e, 'f) + .select("bar".as("d"), "e".attr, "f".attr) .select("bar".as("n")) .select("bar".as("n"), "dummy")).analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReassignLambdaVariableIDSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReassignLambdaVariableIDSuite.scala index 06a32c77ac5e..d454a9b2a634 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReassignLambdaVariableIDSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReassignLambdaVariableIDSuite.scala @@ -32,7 +32,7 @@ class ReassignLambdaVariableIDSuite extends PlanTest { } test("basic: replace positive IDs with unique negative IDs") { - val testRelation = LocalRelation('col.int) + val testRelation = LocalRelation("col".attr.int) val var1 = LambdaVariable("a", BooleanType, true, id = 2) val var2 = LambdaVariable("b", BooleanType, true, id = 4) val query = testRelation.where(var1 && var2) @@ -42,7 +42,7 @@ class ReassignLambdaVariableIDSuite extends PlanTest { } test("ignore LambdaVariable with negative IDs") { - val testRelation = LocalRelation('col.int) + val testRelation = LocalRelation("col".attr.int) val var1 = LambdaVariable("a", BooleanType, true, id = -2) val var2 = LambdaVariable("b", BooleanType, true, id = -4) val query = testRelation.where(var1 && var2) @@ -51,7 +51,7 @@ class ReassignLambdaVariableIDSuite extends PlanTest { } test("fail if positive ID LambdaVariable and negative LambdaVariable both exist") { - val testRelation = LocalRelation('col.int) + val testRelation = LocalRelation("col".attr.int) val var1 = LambdaVariable("a", BooleanType, true, id = -2) val var2 = LambdaVariable("b", BooleanType, true, id = 4) val query = testRelation.where(var1 && var2) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveNoopOperatorsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveNoopOperatorsSuite.scala index cedd21d2bf52..744096094964 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveNoopOperatorsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveNoopOperatorsSuite.scala @@ -31,12 +31,12 @@ class RemoveNoopOperatorsSuite extends PlanTest { RemoveNoopOperators) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) test("Remove all redundant projections in one iteration") { val originalQuery = testRelation - .select('a, 'b, 'c) - .select('a, 'b, 'c) + .select("a".attr, "b".attr, "c".attr) + .select("a".attr, "b".attr, "c".attr) .analyze val optimized = Optimize.execute(originalQuery.analyze) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveNoopUnionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveNoopUnionSuite.scala index 1b40280408e6..6c3a0f245191 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveNoopUnionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveNoopUnionSuite.scala @@ -33,8 +33,9 @@ class RemoveNoopUnionSuite extends PlanTest { RemoveNoopUnion) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int) - val testRelation2 = LocalRelation(output = Seq('a.int, 'b.int), data = Seq(InternalRow(1, 2))) + val testRelation = LocalRelation("a".attr.int, "b".attr.int) + val testRelation2 = + LocalRelation(output = Seq("a".attr.int, "b".attr.int), data = Seq(InternalRow(1, 2))) test("SPARK-34474: Remove redundant Union under Distinct") { val union = Union(testRelation :: testRelation :: Nil) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala index 2e0ab7f64f4d..f91d8fb2febc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala @@ -37,94 +37,99 @@ class RemoveRedundantAliasAndProjectSuite extends PlanTest with PredicateHelper } test("all expressions in project list are aliased child output") { - val relation = LocalRelation('a.int, 'b.int) - val query = relation.select('a as 'a, 'b as 'b).analyze + val relation = LocalRelation("a".attr.int, "b".attr.int) + val query = relation.select("a".attr as "a", "b".attr as "b").analyze val optimized = Optimize.execute(query) comparePlans(optimized, relation) } test("all expressions in project list are aliased child output but with different order") { - val relation = LocalRelation('a.int, 'b.int) - val query = relation.select('b as 'b, 'a as 'a).analyze + val relation = LocalRelation("a".attr.int, "b".attr.int) + val query = relation.select("b".attr as "b", "a".attr as "a").analyze val optimized = Optimize.execute(query) - val expected = relation.select('b, 'a).analyze + val expected = relation.select("b".attr, "a".attr).analyze comparePlans(optimized, expected) } test("some expressions in project list are aliased child output") { - val relation = LocalRelation('a.int, 'b.int) - val query = relation.select('a as 'a, 'b).analyze + val relation = LocalRelation("a".attr.int, "b".attr.int) + val query = relation.select("a".attr as "a", "b".attr).analyze val optimized = Optimize.execute(query) comparePlans(optimized, relation) } test("some expressions in project list are aliased child output but with different order") { - val relation = LocalRelation('a.int, 'b.int) - val query = relation.select('b as 'b, 'a).analyze + val relation = LocalRelation("a".attr.int, "b".attr.int) + val query = relation.select("b".attr as "b", "a".attr).analyze val optimized = Optimize.execute(query) - val expected = relation.select('b, 'a).analyze + val expected = relation.select("b".attr, "a".attr).analyze comparePlans(optimized, expected) } test("some expressions in project list are not Alias or Attribute") { - val relation = LocalRelation('a.int, 'b.int) - val query = relation.select('a as 'a, 'b + 1).analyze + val relation = LocalRelation("a".attr.int, "b".attr.int) + val query = relation.select("a".attr as "a", "b".attr + 1).analyze val optimized = Optimize.execute(query) - val expected = relation.select('a, 'b + 1).analyze + val expected = relation.select("a".attr, "b".attr + 1).analyze comparePlans(optimized, expected) } test("some expressions in project list are aliased child output but with metadata") { - val relation = LocalRelation('a.int, 'b.int) + val relation = LocalRelation("a".attr.int, "b".attr.int) val metadata = new MetadataBuilder().putString("x", "y").build() - val aliasWithMeta = Alias('a, "a")(explicitMetadata = Some(metadata)) - val query = relation.select(aliasWithMeta, 'b).analyze + val aliasWithMeta = Alias("a".attr, "a")(explicitMetadata = Some(metadata)) + val query = relation.select(aliasWithMeta, "b".attr).analyze val optimized = Optimize.execute(query) comparePlans(optimized, query) } test("retain deduplicating alias in self-join") { - val relation = LocalRelation('a.int) - val fragment = relation.select('a as 'a) - val query = fragment.select('a as 'a).join(fragment.select('a as 'a)).analyze + val relation = LocalRelation("a".attr.int) + val fragment = relation.select("a".attr as "a") + val query = fragment.select("a".attr as "a").join(fragment.select("a".attr as "a")).analyze val optimized = Optimize.execute(query) - val expected = relation.join(relation.select('a as 'a)).analyze + val expected = relation.join(relation.select("a".attr as "a")).analyze comparePlans(optimized, expected) } test("alias removal should not break after push project through union") { - val r1 = LocalRelation('a.int) - val r2 = LocalRelation('b.int) - val query = r1.select('a as 'a).union(r2.select('b as 'b)).select('a).analyze + val r1 = LocalRelation("a".attr.int) + val r2 = LocalRelation("b".attr.int) + val query = + r1.select("a".attr as "a").union(r2.select("b".attr as "b")).select("a".attr).analyze val optimized = Optimize.execute(query) val expected = r1.union(r2) comparePlans(optimized, expected) } test("remove redundant alias from aggregate") { - val relation = LocalRelation('a.int, 'b.int) - val query = relation.groupBy('a as 'a)('a as 'a, sum('b)).analyze + val relation = LocalRelation("a".attr.int, "b".attr.int) + val query = relation.groupBy("a".attr as "a")("a".attr as "a", sum("b".attr)).analyze val optimized = Optimize.execute(query) - val expected = relation.groupBy('a)('a, sum('b)).analyze + val expected = relation.groupBy("a".attr)("a".attr, sum("b".attr)).analyze comparePlans(optimized, expected) } test("remove redundant alias from window") { - val relation = LocalRelation('a.int, 'b.int) - val query = relation.window(Seq('b as 'b), Seq('a as 'a), Seq()).analyze + val relation = LocalRelation("a".attr.int, "b".attr.int) + val query = relation.window(Seq("b".attr as "b"), Seq("a".attr as "a"), Seq()).analyze val optimized = Optimize.execute(query) - val expected = relation.window(Seq('b), Seq('a), Seq()).analyze + val expected = relation.window(Seq("b".attr), Seq("a".attr), Seq()).analyze comparePlans(optimized, expected) } test("do not remove output attributes from a subquery") { - val relation = LocalRelation('a.int, 'b.int) + val relation = LocalRelation("a".attr.int, "b".attr.int) val query = Subquery( - relation.select('a as "a", 'b as "b").where('b < 10).select('a).analyze, + relation. + select("a".attr as "a", "b".attr as "b") + .where("b".attr < 10) + .select("a".attr) + .analyze, correlated = false) val optimized = Optimize.execute(query) val expected = Subquery( - relation.select('a as "a", 'b).where('b < 10).select('a).analyze, + relation.select("a".attr as "a", "b".attr).where("b".attr < 10).select("a".attr).analyze, correlated = false) comparePlans(optimized, expected) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala index a1ab0a834474..23cdc7806c59 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala @@ -32,17 +32,17 @@ class ReorderAssociativeOperatorSuite extends PlanTest { ReorderAssociativeOperator) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) test("Reorder associative operators") { val originalQuery = testRelation .select( - (Literal(3) + ((Literal(1) + 'a) + 2)) + 4, - 'b * 1 * 2 * 3 * 4, - ('b + 1) * 2 * 3 * 4, - 'a + 1 + 'b + 2 + 'c + 3, - 'a + 1 + 'b * 2 + 'c + 3, + (Literal(3) + ((Literal(1) + "a".attr) + 2)) + 4, + "b".attr * 1 * 2 * 3 * 4, + ("b".attr + 1) * 2 * 3 * 4, + "a".attr + 1 + "b".attr + 2 + "c".attr + 3, + "a".attr + 1 + "b".attr * 2 + "c".attr + 3, Rand(0) * 1 * 2 * 3 * 4) val optimized = Optimize.execute(originalQuery.analyze) @@ -50,11 +50,11 @@ class ReorderAssociativeOperatorSuite extends PlanTest { val correctAnswer = testRelation .select( - ('a + 10).as("((3 + ((1 + a) + 2)) + 4)"), - ('b * 24).as("((((b * 1) * 2) * 3) * 4)"), - (('b + 1) * 24).as("((((b + 1) * 2) * 3) * 4)"), - ('a + 'b + 'c + 6).as("(((((a + 1) + b) + 2) + c) + 3)"), - ('a + 'b * 2 + 'c + 4).as("((((a + 1) + (b * 2)) + c) + 3)"), + ("a".attr + 10).as("((3 + ((1 + a) + 2)) + 4)"), + ("b".attr * 24).as("((((b * 1) * 2) * 3) * 4)"), + (("b".attr + 1) * 24).as("((((b + 1) * 2) * 3) * 4)"), + ("a".attr + "b".attr + "c".attr + 6).as("(((((a + 1) + b) + 2) + c) + 3)"), + ("a".attr + "b".attr * 2 + "c".attr + 4).as("((((a + 1) + (b * 2)) + c) + 3)"), Rand(0) * 1 * 2 * 3 * 4) .analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicateSuite.scala index 5183cca1ebf8..65f9f3b6e9e4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicateSuite.scala @@ -42,8 +42,9 @@ class ReplaceNullWithFalseInPredicateSuite extends PlanTest { } private val testRelation = - LocalRelation('i.int, 'b.boolean, 'a.array(IntegerType), 'm.map(IntegerType, IntegerType)) - private val anotherTestRelation = LocalRelation('d.int) + LocalRelation("i".attr.int, "b".attr.boolean, "a".attr.array(IntegerType), + "m".attr.mapAttr(IntegerType, IntegerType)) + private val anotherTestRelation = LocalRelation("d".attr.int) test("replace null inside filter and join conditions") { testFilter(originalCond = Literal(null, BooleanType), expectedCond = FalseLiteral) @@ -365,36 +366,36 @@ class ReplaceNullWithFalseInPredicateSuite extends PlanTest { testProjection(originalExpr = column, expectedExpr = column) } - private def lv(s: Symbol) = UnresolvedNamedLambdaVariable(Seq(s.name)) + private def lv(s: String) = UnresolvedNamedLambdaVariable(Seq(s)) test("replace nulls in lambda function of ArrayFilter") { - testHigherOrderFunc('a, ArrayFilter, Seq(lv('e))) + testHigherOrderFunc("a".attr, ArrayFilter, Seq(lv("e"))) } test("replace nulls in lambda function of ArrayExists") { withSQLConf(SQLConf.LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC.key -> "true") { - val lambdaArgs = Seq(lv('e)) + val lambdaArgs = Seq(lv("e")) val cond = GreaterThan(lambdaArgs.last, Literal(0)) val lambda = LambdaFunction( function = If(cond, Literal(null, BooleanType), TrueLiteral), arguments = lambdaArgs) - val expr = ArrayExists('a, lambda) + val expr = ArrayExists("a".attr, lambda) testProjection(originalExpr = expr, expectedExpr = expr) } withSQLConf(SQLConf.LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC.key -> "false") { - testHigherOrderFunc('a, ArrayExists.apply, Seq(lv('e))) + testHigherOrderFunc("a".attr, ArrayExists.apply, Seq(lv("e"))) } } test("replace nulls in lambda function of MapFilter") { - testHigherOrderFunc('m, MapFilter, Seq(lv('k), lv('v))) + testHigherOrderFunc("m".attr, MapFilter, Seq(lv("k"), lv("v"))) } test("inability to replace nulls in arbitrary higher-order function") { val lambdaFunc = LambdaFunction( - function = If(lv('e) > 0, Literal(null, BooleanType), TrueLiteral), - arguments = Seq[NamedExpression](lv('e))) - val column = ArrayTransform('a, lambdaFunc) + function = If(lv("e") > 0, Literal(null, BooleanType), TrueLiteral), + arguments = Seq[NamedExpression](lv("e"))) + val column = ArrayTransform("a".attr, lambdaFunc) testProjection(originalExpr = column, expectedExpr = column) } @@ -457,10 +458,10 @@ class ReplaceNullWithFalseInPredicateSuite extends PlanTest { private def testMerge(originalCond: Expression, expectedCond: Expression): Unit = { val func = (rel: LogicalPlan, expr: Expression) => { val assignments = Seq( - Assignment('i, 'i), - Assignment('b, 'b), - Assignment('a, 'a), - Assignment('m, 'm) + Assignment("i".attr, "i".attr), + Assignment("b".attr, "b".attr), + Assignment("a".attr, "a".attr), + Assignment("m".attr, "m".attr) ) val matchedActions = UpdateAction(Some(expr), assignments) :: DeleteAction(Some(expr)) :: Nil val notMatchedActions = InsertAction(Some(expr), assignments) :: Nil @@ -484,8 +485,8 @@ class ReplaceNullWithFalseInPredicateSuite extends PlanTest { function = !(cond <=> TrueLiteral), arguments = lambdaArgs) testProjection( - originalExpr = createExpr(argument, lambda1) as 'x, - expectedExpr = createExpr(argument, lambda2) as 'x) + originalExpr = createExpr(argument, lambda1) as "x", + expectedExpr = createExpr(argument, lambda2) as "x") } private def test( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala index 9bf864f5201f..1c362e44e582 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala @@ -40,22 +40,23 @@ class ReplaceOperatorSuite extends PlanTest { } test("replace Intersect with Left-semi Join") { - val table1 = LocalRelation('a.int, 'b.int) - val table2 = LocalRelation('c.int, 'd.int) + val table1 = LocalRelation("a".attr.int, "b".attr.int) + val table2 = LocalRelation("c".attr.int, "d".attr.int) val query = Intersect(table1, table2, isAll = false) val optimized = Optimize.execute(query.analyze) val correctAnswer = Aggregate(table1.output, table1.output, - Join(table1, table2, LeftSemi, Option('a <=> 'c && 'b <=> 'd), JoinHint.NONE)).analyze + Join(table1, table2, LeftSemi, + Option("a".attr <=> "c".attr && "b".attr <=> "d".attr), JoinHint.NONE)).analyze comparePlans(optimized, correctAnswer) } test("replace Except with Filter while both the nodes are of type Filter") { - val attributeA = 'a.int - val attributeB = 'b.int + val attributeA = "a".attr.int + val attributeB = "b".attr.int val table1 = LocalRelation.fromExternalRows(Seq(attributeA, attributeB), data = Seq(Row(1, 2))) val table2 = Filter(attributeB === 2, Filter(attributeA === 1, table1)) @@ -73,8 +74,8 @@ class ReplaceOperatorSuite extends PlanTest { } test("replace Except with Filter while only right node is of type Filter") { - val attributeA = 'a.int - val attributeB = 'b.int + val attributeA = "a".attr.int + val attributeB = "b".attr.int val table1 = LocalRelation.fromExternalRows(Seq(attributeA, attributeB), data = Seq(Row(1, 2))) val table2 = Filter(attributeB < 1, Filter(attributeA >= 2, table1)) @@ -91,8 +92,8 @@ class ReplaceOperatorSuite extends PlanTest { } test("replace Except with Filter while both the nodes are of type Project") { - val attributeA = 'a.int - val attributeB = 'b.int + val attributeA = "a".attr.int + val attributeB = "b".attr.int val table1 = LocalRelation.fromExternalRows(Seq(attributeA, attributeB), data = Seq(Row(1, 2))) val table2 = Project(Seq(attributeA, attributeB), table1) @@ -111,8 +112,8 @@ class ReplaceOperatorSuite extends PlanTest { } test("replace Except with Filter while only right node is of type Project") { - val attributeA = 'a.int - val attributeB = 'b.int + val attributeA = "a".attr.int + val attributeB = "b".attr.int val table1 = LocalRelation.fromExternalRows(Seq(attributeA, attributeB), data = Seq(Row(1, 2))) val table2 = Filter(attributeB === 2, Filter(attributeA === 1, table1)) @@ -131,8 +132,8 @@ class ReplaceOperatorSuite extends PlanTest { } test("replace Except with Filter while left node is Project and right node is Filter") { - val attributeA = 'a.int - val attributeB = 'b.int + val attributeA = "a".attr.int + val attributeB = "b".attr.int val table1 = LocalRelation.fromExternalRows(Seq(attributeA, attributeB), data = Seq(Row(1, 2))) val table2 = Project(Seq(attributeA, attributeB), @@ -152,23 +153,24 @@ class ReplaceOperatorSuite extends PlanTest { } test("replace Except with Left-anti Join") { - val table1 = LocalRelation('a.int, 'b.int) - val table2 = LocalRelation('c.int, 'd.int) + val table1 = LocalRelation("a".attr.int, "b".attr.int) + val table2 = LocalRelation("c".attr.int, "d".attr.int) val query = Except(table1, table2, isAll = false) val optimized = Optimize.execute(query.analyze) val correctAnswer = Aggregate(table1.output, table1.output, - Join(table1, table2, LeftAnti, Option('a <=> 'c && 'b <=> 'd), JoinHint.NONE)).analyze + Join(table1, table2, LeftAnti, + Option("a".attr <=> "c".attr && "b".attr <=> "d".attr), JoinHint.NONE)).analyze comparePlans(optimized, correctAnswer) } test("replace Except with Filter when only right filter can be applied to the left") { - val table = LocalRelation(Seq('a.int, 'b.int)) - val left = table.where('b < 1).select('a).as("left") - val right = table.where('b < 3).select('a).as("right") + val table = LocalRelation(Seq("a".attr.int, "b".attr.int)) + val left = table.where("b".attr < 1).select("a".attr).as("left") + val right = table.where("b".attr < 3).select("a".attr).as("right") val query = Except(left, right, isAll = false) val optimized = Optimize.execute(query.analyze) @@ -181,7 +183,7 @@ class ReplaceOperatorSuite extends PlanTest { } test("replace Distinct with Aggregate") { - val input = LocalRelation('a.int, 'b.int) + val input = LocalRelation("a".attr.int, "b".attr.int) val query = Distinct(input) val optimized = Optimize.execute(query.analyze) @@ -192,7 +194,7 @@ class ReplaceOperatorSuite extends PlanTest { } test("replace batch Deduplicate with Aggregate") { - val input = LocalRelation('a.int, 'b.int) + val input = LocalRelation("a".attr.int, "b".attr.int) val attrA = input.output(0) val attrB = input.output(1) val query = Deduplicate(Seq(attrA), input) // dropDuplicates("a") @@ -219,7 +221,7 @@ class ReplaceOperatorSuite extends PlanTest { } test("don't replace streaming Deduplicate") { - val input = LocalRelation(Seq('a.int, 'b.int), isStreaming = true) + val input = LocalRelation(Seq("a".attr.int, "b".attr.int), isStreaming = true) val attrA = input.output(0) val query = Deduplicate(Seq(attrA), input) // dropDuplicates("a") val optimized = Optimize.execute(query.analyze) @@ -228,21 +230,21 @@ class ReplaceOperatorSuite extends PlanTest { } test("SPARK-26366: ReplaceExceptWithFilter should handle properly NULL") { - val basePlan = LocalRelation(Seq('a.int, 'b.int)) - val otherPlan = basePlan.where('a.in(1, 2) || 'b.in()) + val basePlan = LocalRelation(Seq("a".attr.int, "b".attr.int)) + val otherPlan = basePlan.where("a".attr.in(1, 2) || "b".attr.in()) val except = Except(basePlan, otherPlan, false) val result = OptimizeIn(Optimize.execute(except.analyze)) val correctAnswer = Aggregate(basePlan.output, basePlan.output, - Filter(!Coalesce(Seq( - 'a.in(1, 2) || If('b.isNotNull, Literal.FalseLiteral, Literal(null, BooleanType)), + Filter(!Coalesce(Seq("a".attr.in(1, 2) || If("b".attr.isNotNull, + Literal.FalseLiteral, Literal(null, BooleanType)), Literal.FalseLiteral)), basePlan)).analyze comparePlans(result, correctAnswer) } test("SPARK-26366: ReplaceExceptWithFilter should not transform non-deterministic") { - val basePlan = LocalRelation(Seq('a.int, 'b.int)) - val otherPlan = basePlan.where('a > rand(1L)) + val basePlan = LocalRelation(Seq("a".attr.int, "b".attr.int)) + val otherPlan = basePlan.where("a".attr > rand(1L)) val except = Except(basePlan, otherPlan, false) val result = Optimize.execute(except.analyze) val condition = basePlan.output.zip(otherPlan.output).map { case (a1, a2) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala index 5d6abf516f28..e1fdc70734b1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala @@ -27,7 +27,8 @@ import org.apache.spark.sql.types.{IntegerType, StringType} class RewriteDistinctAggregatesSuite extends PlanTest { val nullInt = Literal(null, IntegerType) val nullString = Literal(null, StringType) - val testRelation = LocalRelation('a.string, 'b.string, 'c.string, 'd.string, 'e.int) + val testRelation = + LocalRelation("a".attr.string, "b".attr.string, "c".attr.string, "d".attr.string, "e".attr.int) private def checkRewrite(rewrite: LogicalPlan): Unit = rewrite match { case Aggregate(_, _, Aggregate(_, _, _: Expand)) => @@ -36,7 +37,7 @@ class RewriteDistinctAggregatesSuite extends PlanTest { test("single distinct group") { val input = testRelation - .groupBy('a)(countDistinct('e)) + .groupBy("a".attr)(countDistinct("e".attr)) .analyze val rewrite = RewriteDistinctAggregates(input) comparePlans(input, rewrite) @@ -44,9 +45,9 @@ class RewriteDistinctAggregatesSuite extends PlanTest { test("single distinct group with partial aggregates") { val input = testRelation - .groupBy('a, 'd)( - countDistinct('e, 'c).as('agg1), - max('b).as('agg2)) + .groupBy("a".attr, "d".attr)( + countDistinct("e".attr, "c".attr).as("agg1"), + max("b".attr).as("agg2")) .analyze val rewrite = RewriteDistinctAggregates(input) comparePlans(input, rewrite) @@ -54,24 +55,24 @@ class RewriteDistinctAggregatesSuite extends PlanTest { test("multiple distinct groups") { val input = testRelation - .groupBy('a)(countDistinct('b, 'c), countDistinct('d)) + .groupBy("a".attr)(countDistinct("b".attr, "c".attr), countDistinct("d".attr)) .analyze checkRewrite(RewriteDistinctAggregates(input)) } test("multiple distinct groups with partial aggregates") { val input = testRelation - .groupBy('a)(countDistinct('b, 'c), countDistinct('d), sum('e)) + .groupBy("a".attr)(countDistinct("b".attr, "c".attr), countDistinct("d".attr), sum("e".attr)) .analyze checkRewrite(RewriteDistinctAggregates(input)) } test("multiple distinct groups with non-partial aggregates") { val input = testRelation - .groupBy('a)( - countDistinct('b, 'c), - countDistinct('d), - CollectSet('b).toAggregateExpression()) + .groupBy("a".attr)( + countDistinct("b".attr, "c".attr), + countDistinct("d".attr), + CollectSet("b".attr).toAggregateExpression()) .analyze checkRewrite(RewriteDistinctAggregates(input)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala index 0608ded73937..6d4cfebfea0b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteSubquerySuite.scala @@ -38,30 +38,34 @@ class RewriteSubquerySuite extends PlanTest { } test("Column pruning after rewriting predicate subquery") { - val relation = LocalRelation('a.int, 'b.int) - val relInSubquery = LocalRelation('x.int, 'y.int, 'z.int) + val relation = LocalRelation("a".attr.int, "b".attr.int) + val relInSubquery = LocalRelation("x".attr.int, "y".attr.int, "z".attr.int) - val query = relation.where('a.in(ListQuery(relInSubquery.select('x)))).select('a) + val query = + relation.where("a".attr.in(ListQuery(relInSubquery.select("x".attr)))).select("a".attr) val optimized = Optimize.execute(query.analyze) val correctAnswer = relation - .select('a) - .join(relInSubquery.select('x), LeftSemi, Some('a === 'x)) + .select("a".attr) + .join(relInSubquery.select("x".attr), LeftSemi, Some("a".attr === "x".attr)) .analyze comparePlans(optimized, correctAnswer) } test("NOT-IN subquery nested inside OR") { - val relation1 = LocalRelation('a.int, 'b.int) - val relation2 = LocalRelation('c.int, 'd.int) - val exists = 'exists.boolean.notNull + val relation1 = LocalRelation("a".attr.int, "b".attr.int) + val relation2 = LocalRelation("c".attr.int, "d".attr.int) + val exists = "exists".attr.boolean.notNull - val query = relation1.where('b === 1 || Not('a.in(ListQuery(relation2.select('c))))).select('a) + val query = relation1. + where("b".attr === 1 || Not("a".attr.in(ListQuery(relation2.select("c".attr))))) + .select("a".attr) val correctAnswer = relation1 - .join(relation2.select('c), ExistenceJoin(exists), Some('a === 'c || IsNull('a === 'c))) - .where('b === 1 || Not(exists)) - .select('a) + .join(relation2.select("c".attr), ExistenceJoin(exists), + Some("a".attr === "c".attr || IsNull("a".attr === "c".attr))) + .where("b".attr === 1 || Not(exists)) + .select("a".attr) .analyze val optimized = Optimize.execute(query.analyze) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala index 3fa7df3c9494..4e73133bfd1a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala @@ -38,9 +38,9 @@ class SetOperationSuite extends PlanTest { PruneFilters) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) - val testRelation2 = LocalRelation('d.int, 'e.int, 'f.int) - val testRelation3 = LocalRelation('g.int, 'h.int, 'i.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) + val testRelation2 = LocalRelation("d".attr.int, "e".attr.int, "f".attr.int) + val testRelation3 = LocalRelation("g".attr.int, "h".attr.int, "i".attr.int) val testUnion = Union(testRelation :: testRelation2 :: testRelation3 :: Nil) test("union: combine unions into one unions") { @@ -59,33 +59,33 @@ class SetOperationSuite extends PlanTest { } test("union: filter to each side") { - val unionQuery = testUnion.where('a === 1) + val unionQuery = testUnion.where("a".attr === 1) val unionOptimized = Optimize.execute(unionQuery.analyze) val unionCorrectAnswer = - Union(testRelation.where('a === 1) :: - testRelation2.where('d === 1) :: - testRelation3.where('g === 1) :: Nil).analyze + Union(testRelation.where("a".attr === 1) :: + testRelation2.where("d".attr === 1) :: + testRelation3.where("g".attr === 1) :: Nil).analyze comparePlans(unionOptimized, unionCorrectAnswer) } test("union: project to each side") { - val unionQuery = testUnion.select('a) + val unionQuery = testUnion.select("a".attr) val unionOptimized = Optimize.execute(unionQuery.analyze) val unionCorrectAnswer = - Union(testRelation.select('a) :: - testRelation2.select('d) :: - testRelation3.select('g) :: Nil).analyze + Union(testRelation.select("a".attr) :: + testRelation2.select("d".attr) :: + testRelation3.select("g".attr) :: Nil).analyze comparePlans(unionOptimized, unionCorrectAnswer) } test("Remove unnecessary distincts in multiple unions") { val query1 = OneRowRelation() - .select(Literal(1).as('a)) + .select(Literal(1).as("a")) val query2 = OneRowRelation() - .select(Literal(2).as('b)) + .select(Literal(2).as("b")) val query3 = OneRowRelation() - .select(Literal(3).as('c)) + .select(Literal(3).as("c")) // D - U - D - U - query1 // | | @@ -113,13 +113,13 @@ class SetOperationSuite extends PlanTest { test("Keep necessary distincts in multiple unions") { val query1 = OneRowRelation() - .select(Literal(1).as('a)) + .select(Literal(1).as("a")) val query2 = OneRowRelation() - .select(Literal(2).as('b)) + .select(Literal(2).as("b")) val query3 = OneRowRelation() - .select(Literal(3).as('c)) + .select(Literal(3).as("c")) val query4 = OneRowRelation() - .select(Literal(4).as('d)) + .select(Literal(4).as("d")) // U - D - U - query1 // | | @@ -148,11 +148,11 @@ class SetOperationSuite extends PlanTest { test("SPARK-34283: Remove unnecessary deduplicate in multiple unions") { val query1 = OneRowRelation() - .select(Literal(1).as('a)) + .select(Literal(1).as("a")) val query2 = OneRowRelation() - .select(Literal(2).as('b)) + .select(Literal(2).as("b")) val query3 = OneRowRelation() - .select(Literal(3).as('c)) + .select(Literal(3).as("c")) // D - U - D - U - query1 // | | @@ -195,13 +195,13 @@ class SetOperationSuite extends PlanTest { test("SPARK-34283: Keep necessary deduplicate in multiple unions") { val query1 = OneRowRelation() - .select(Literal(1).as('a)) + .select(Literal(1).as("a")) val query2 = OneRowRelation() - .select(Literal(2).as('b)) + .select(Literal(2).as("b")) val query3 = OneRowRelation() - .select(Literal(3).as('c)) + .select(Literal(3).as("c")) val query4 = OneRowRelation() - .select(Literal(4).as('d)) + .select(Literal(4).as("d")) // U - D - U - query1 // | | @@ -238,10 +238,11 @@ class SetOperationSuite extends PlanTest { val input = Except(testRelation, testRelation2, isAll = true) val rewrittenPlan = RewriteExceptAll(input) - val planFragment = testRelation.select(Literal(1L).as("vcol"), 'a, 'b, 'c) - .union(testRelation2.select(Literal(-1L).as("vcol"), 'd, 'e, 'f)) - .groupBy('a, 'b, 'c)('a, 'b, 'c, sum('vcol).as("sum")) - .where(GreaterThan('sum, Literal(0L))).analyze + val planFragment = testRelation.select(Literal(1L).as("vcol"), "a".attr, "b".attr, "c".attr) + .union(testRelation2.select(Literal(-1L).as("vcol"), "d".attr, "e".attr, "f".attr)) + .groupBy("a".attr, "b".attr, "c".attr)("a".attr, "b".attr, "c".attr, + sum("vcol".attr).as("sum")) + .where(GreaterThan("sum".attr, Literal(0L))).analyze val multiplierAttr = planFragment.output.last val output = planFragment.output.dropRight(1) val expectedPlan = Project(output, @@ -260,16 +261,19 @@ class SetOperationSuite extends PlanTest { val input = Intersect(testRelation, testRelation2, isAll = true) val rewrittenPlan = RewriteIntersectAll(input) val leftRelation = testRelation - .select(Literal(true).as("vcol1"), Literal(null, BooleanType).as("vcol2"), 'a, 'b, 'c) + .select(Literal(true).as("vcol1"), + Literal(null, BooleanType).as("vcol2"), "a".attr, "b".attr, "c".attr) val rightRelation = testRelation2 - .select(Literal(null, BooleanType).as("vcol1"), Literal(true).as("vcol2"), 'd, 'e, 'f) + .select(Literal(null, BooleanType).as("vcol1"), + Literal(true).as("vcol2"), "d".attr, "e".attr, "f".attr) val planFragment = leftRelation.union(rightRelation) - .groupBy('a, 'b, 'c)(count('vcol1).as("vcol1_count"), - count('vcol2).as("vcol2_count"), 'a, 'b, 'c) - .where(And(GreaterThanOrEqual('vcol1_count, Literal(1L)), - GreaterThanOrEqual('vcol2_count, Literal(1L)))) - .select('a, 'b, 'c, - If(GreaterThan('vcol1_count, 'vcol2_count), 'vcol2_count, 'vcol1_count).as("min_count")) + .groupBy("a".attr, "b".attr, "c".attr)(count("vcol1".attr).as("vcol1_count"), + count("vcol2".attr).as("vcol2_count"), "a".attr, "b".attr, "c".attr) + .where(And(GreaterThanOrEqual("vcol1_count".attr, Literal(1L)), + GreaterThanOrEqual("vcol2_count".attr, Literal(1L)))) + .select("a".attr, "b".attr, "c".attr, + If(GreaterThan("vcol1_count".attr, "vcol2_count".attr), + "vcol2_count".attr, "vcol1_count".attr).as("min_count")) .analyze val multiplierAttr = planFragment.output.last val output = planFragment.output.dropRight(1) @@ -286,27 +290,27 @@ class SetOperationSuite extends PlanTest { } test("SPARK-23356 union: expressions with literal in project list are pushed down") { - val unionQuery = testUnion.select(('a + 1).as("aa")) + val unionQuery = testUnion.select(("a".attr + 1).as("aa")) val unionOptimized = Optimize.execute(unionQuery.analyze) val unionCorrectAnswer = - Union(testRelation.select(('a + 1).as("aa")) :: - testRelation2.select(('d + 1).as("aa")) :: - testRelation3.select(('g + 1).as("aa")) :: Nil).analyze + Union(testRelation.select(("a".attr + 1).as("aa")) :: + testRelation2.select(("d".attr + 1).as("aa")) :: + testRelation3.select(("g".attr + 1).as("aa")) :: Nil).analyze comparePlans(unionOptimized, unionCorrectAnswer) } test("SPARK-23356 union: expressions in project list are pushed down") { - val unionQuery = testUnion.select(('a + 'b).as("ab")) + val unionQuery = testUnion.select(("a".attr + "b".attr).as("ab")) val unionOptimized = Optimize.execute(unionQuery.analyze) val unionCorrectAnswer = - Union(testRelation.select(('a + 'b).as("ab")) :: - testRelation2.select(('d + 'e).as("ab")) :: - testRelation3.select(('g + 'h).as("ab")) :: Nil).analyze + Union(testRelation.select(("a".attr + "b".attr).as("ab")) :: + testRelation2.select(("d".attr + "e".attr).as("ab")) :: + testRelation3.select(("g".attr + "h".attr).as("ab")) :: Nil).analyze comparePlans(unionOptimized, unionCorrectAnswer) } test("SPARK-23356 union: no pushdown for non-deterministic expression") { - val unionQuery = testUnion.select('a, Rand(10).as("rnd")) + val unionQuery = testUnion.select("a".attr, Rand(10).as("rnd")) val unionOptimized = Optimize.execute(unionQuery.analyze) val unionCorrectAnswer = unionQuery.analyze comparePlans(unionOptimized, unionCorrectAnswer) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCastsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCastsSuite.scala index c981cee55d0f..3c2aa948527d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCastsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCastsSuite.scala @@ -31,15 +31,15 @@ class SimplifyCastsSuite extends PlanTest { } test("non-nullable element array to nullable element array cast") { - val input = LocalRelation('a.array(ArrayType(IntegerType, false))) - val plan = input.select('a.cast(ArrayType(IntegerType, true)).as("casted")).analyze + val input = LocalRelation("a".attr.array(ArrayType(IntegerType, false))) + val plan = input.select("a".attr.cast(ArrayType(IntegerType, true)).as("casted")).analyze val optimized = Optimize.execute(plan) - val expected = input.select('a.as("casted")).analyze + val expected = input.select("a".attr.as("casted")).analyze comparePlans(optimized, expected) } test("nullable element to non-nullable element array cast") { - val input = LocalRelation('a.array(ArrayType(IntegerType, true))) + val input = LocalRelation("a".attr.array(ArrayType(IntegerType, true))) val attr = input.output.head val plan = input.select(attr.cast(ArrayType(IntegerType, false)).as("casted")) val optimized = Optimize.execute(plan) @@ -49,16 +49,16 @@ class SimplifyCastsSuite extends PlanTest { } test("non-nullable value map to nullable value map cast") { - val input = LocalRelation('m.map(MapType(StringType, StringType, false))) - val plan = input.select('m.cast(MapType(StringType, StringType, true)) + val input = LocalRelation("m".attr.mapAttr(MapType(StringType, StringType, false))) + val plan = input.select("m".attr.cast(MapType(StringType, StringType, true)) .as("casted")).analyze val optimized = Optimize.execute(plan) - val expected = input.select('m.as("casted")).analyze + val expected = input.select("m".attr.as("casted")).analyze comparePlans(optimized, expected) } test("nullable value map to non-nullable value map cast") { - val input = LocalRelation('m.map(MapType(StringType, StringType, true))) + val input = LocalRelation("m".attr.mapAttr(MapType(StringType, StringType, true))) val attr = input.output.head val plan = input.select(attr.cast(MapType(StringType, StringType, false)) .as("casted")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala index 2a685bfeefcb..e29b556a6cd4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala @@ -35,7 +35,7 @@ class SimplifyConditionalSuite extends PlanTest with ExpressionEvalHelper with P BooleanSimplification, ConstantFolding, SimplifyConditionals) :: Nil } - private val relation = LocalRelation('a.int, 'b.int, 'c.boolean) + private val relation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.boolean) protected def assertEquivalent(e1: Expression, e2: Expression): Unit = { val correctAnswer = Project(Alias(e2, "out")() :: Nil, relation).analyze @@ -126,9 +126,9 @@ class SimplifyConditionalSuite extends PlanTest with ExpressionEvalHelper with P test("simplify CaseWhen if all the outputs are semantic equivalence") { // When the conditions in `CaseWhen` are all deterministic, `CaseWhen` can be removed. assertEquivalent( - CaseWhen(('a.isNotNull, Subtract(Literal(3), Literal(2))) :: - ('b.isNull, Literal(1)) :: - (!'c, Add(Literal(6), Literal(-5))) :: + CaseWhen(("a".attr.isNotNull, Subtract(Literal(3), Literal(2))) :: + ("b".attr.isNull, Literal(1)) :: + (!"c".attr, Add(Literal(6), Literal(-5))) :: Nil, Add(Literal(2), Literal(-1))), Literal(1) @@ -167,19 +167,19 @@ class SimplifyConditionalSuite extends PlanTest with ExpressionEvalHelper with P } test("simplify if when one clause is null and another is boolean") { - val p = IsNull('a) + val p = IsNull("a".attr) val nullLiteral = Literal(null, BooleanType) assertEquivalent(If(p, nullLiteral, FalseLiteral), And(p, nullLiteral)) - assertEquivalent(If(p, nullLiteral, TrueLiteral), Or(IsNotNull('a), nullLiteral)) - assertEquivalent(If(p, FalseLiteral, nullLiteral), And(IsNotNull('a), nullLiteral)) + assertEquivalent(If(p, nullLiteral, TrueLiteral), Or(IsNotNull("a".attr), nullLiteral)) + assertEquivalent(If(p, FalseLiteral, nullLiteral), And(IsNotNull("a".attr), nullLiteral)) assertEquivalent(If(p, TrueLiteral, nullLiteral), Or(p, nullLiteral)) // the rule should not apply to nullable predicate Seq(TrueLiteral, FalseLiteral).foreach { b => - assertEquivalent(If(GreaterThan('a, 42), nullLiteral, b), - If(GreaterThan('a, 42), nullLiteral, b)) - assertEquivalent(If(GreaterThan('a, 42), b, nullLiteral), - If(GreaterThan('a, 42), b, nullLiteral)) + assertEquivalent(If(GreaterThan("a".attr, 42), nullLiteral, b), + If(GreaterThan("a".attr, 42), nullLiteral, b)) + assertEquivalent(If(GreaterThan("a".attr, 42), b, nullLiteral), + If(GreaterThan("a".attr, 42), b, nullLiteral)) } // check evaluation also @@ -203,10 +203,10 @@ class SimplifyConditionalSuite extends PlanTest with ExpressionEvalHelper with P test("SPARK-33845: remove unnecessary if when the outputs are boolean type") { // verify the boolean equivalence of all transformations involved val fields = Seq( - 'cond.boolean.notNull, - 'cond_nullable.boolean, - 'a.boolean, - 'b.boolean + "cond".attr.boolean.notNull, + "cond_nullable".attr.boolean, + "a".attr.boolean, + "b".attr.boolean ) val Seq(cond, cond_nullable, a, b) = fields.zipWithIndex.map { case (f, i) => f.at(i) } @@ -238,7 +238,7 @@ class SimplifyConditionalSuite extends PlanTest with ExpressionEvalHelper with P test("SPARK-33847: Remove the CaseWhen if elseValue is empty and other outputs are null") { assertEquivalent( - CaseWhen((GreaterThan('a, 1), Literal.create(null, IntegerType)) :: Nil, None), + CaseWhen((GreaterThan("a".attr, 1), Literal.create(null, IntegerType)) :: Nil, None), Literal.create(null, IntegerType)) assertEquivalent( @@ -249,10 +249,10 @@ class SimplifyConditionalSuite extends PlanTest with ExpressionEvalHelper with P test("SPARK-33884: simplify CaseWhen clauses with (true and false) and (false and true)") { // verify the boolean equivalence of all transformations involved val fields = Seq( - 'cond.boolean.notNull, - 'cond_nullable.boolean, - 'a.boolean, - 'b.boolean + "cond".attr.boolean.notNull, + "cond_nullable".attr.boolean, + "a".attr.boolean, + "b".attr.boolean ) val Seq(cond, cond_nullable, a, b) = fields.zipWithIndex.map { case (f, i) => f.at(i) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicateSuite.scala index 04ebb4e63c67..d7faf51eca3f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicateSuite.scala @@ -41,8 +41,9 @@ class SimplifyConditionalsInPredicateSuite extends PlanTest { } private val testRelation = - LocalRelation('i.int, 'b.boolean, 'a.array(IntegerType), 'm.map(IntegerType, IntegerType)) - private val anotherTestRelation = LocalRelation('d.int) + LocalRelation("i".attr.int, "b".attr.boolean, "a".attr.array(IntegerType), + "m".attr.mapAttr(IntegerType, IntegerType)) + private val anotherTestRelation = LocalRelation("d".attr.int) test("IF(cond, trueVal, false) => AND(cond, trueVal)") { val originalCond = If( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyStringCaseConversionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyStringCaseConversionSuite.scala index b9bf930f0ea0..9b3b91f24717 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyStringCaseConversionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyStringCaseConversionSuite.scala @@ -32,17 +32,17 @@ class SimplifyStringCaseConversionSuite extends PlanTest { SimplifyCaseConversionExpressions) :: Nil } - val testRelation = LocalRelation('a.string) + val testRelation = LocalRelation("a".attr.string) test("simplify UPPER(UPPER(str))") { val originalQuery = testRelation - .select(Upper(Upper('a)) as 'u) + .select(Upper(Upper("a".attr)) as "u") val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select(Upper('a) as 'u) + .select(Upper("a".attr) as "u") .analyze comparePlans(optimized, correctAnswer) @@ -51,12 +51,12 @@ class SimplifyStringCaseConversionSuite extends PlanTest { test("simplify UPPER(LOWER(str))") { val originalQuery = testRelation - .select(Upper(Lower('a)) as 'u) + .select(Upper(Lower("a".attr)) as "u") val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select(Upper('a) as 'u) + .select(Upper("a".attr) as "u") .analyze comparePlans(optimized, correctAnswer) @@ -65,11 +65,11 @@ class SimplifyStringCaseConversionSuite extends PlanTest { test("simplify LOWER(UPPER(str))") { val originalQuery = testRelation - .select(Lower(Upper('a)) as 'l) + .select(Lower(Upper("a".attr)) as "l") val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select(Lower('a) as 'l) + .select(Lower("a".attr) as "l") .analyze comparePlans(optimized, correctAnswer) @@ -78,11 +78,11 @@ class SimplifyStringCaseConversionSuite extends PlanTest { test("simplify LOWER(LOWER(str))") { val originalQuery = testRelation - .select(Lower(Lower('a)) as 'l) + .select(Lower(Lower("a".attr)) as "l") val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .select(Lower('a) as 'l) + .select(Lower("a".attr) as "l") .analyze comparePlans(optimized, correctAnswer) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TransposeWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TransposeWindowSuite.scala index 4acd57832d2f..506363d0fa0e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TransposeWindowSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TransposeWindowSuite.scala @@ -31,7 +31,7 @@ class TransposeWindowSuite extends PlanTest { Batch("FlipWindow", Once, CollapseWindow, TransposeWindow) :: Nil } - val testRelation = LocalRelation('a.string, 'b.string, 'c.int, 'd.string) + val testRelation = LocalRelation("a".attr.string, "b".attr.string, "c".attr.int, "d".attr.string) val a = testRelation.output(0) val b = testRelation.output(1) @@ -48,40 +48,40 @@ class TransposeWindowSuite extends PlanTest { test("transpose two adjacent windows with compatible partitions") { val query = testRelation - .window(Seq(sum(c).as('sum_a_2)), partitionSpec2, orderSpec2) - .window(Seq(sum(c).as('sum_a_1)), partitionSpec1, orderSpec1) + .window(Seq(sum(c).as("sum_a_2")), partitionSpec2, orderSpec2) + .window(Seq(sum(c).as("sum_a_1")), partitionSpec1, orderSpec1) val analyzed = query.analyze val optimized = Optimize.execute(analyzed) val correctAnswer = testRelation - .window(Seq(sum(c).as('sum_a_1)), partitionSpec1, orderSpec1) - .window(Seq(sum(c).as('sum_a_2)), partitionSpec2, orderSpec2) - .select('a, 'b, 'c, 'd, 'sum_a_2, 'sum_a_1) + .window(Seq(sum(c).as("sum_a_1")), partitionSpec1, orderSpec1) + .window(Seq(sum(c).as("sum_a_2")), partitionSpec2, orderSpec2) + .select("a".attr, "b".attr, "c".attr, "d".attr, "sum_a_2".attr, "sum_a_1".attr) comparePlans(optimized, correctAnswer.analyze) } test("transpose two adjacent windows with differently ordered compatible partitions") { val query = testRelation - .window(Seq(sum(c).as('sum_a_2)), partitionSpec4, Seq.empty) - .window(Seq(sum(c).as('sum_a_1)), partitionSpec2, Seq.empty) + .window(Seq(sum(c).as("sum_a_2")), partitionSpec4, Seq.empty) + .window(Seq(sum(c).as("sum_a_1")), partitionSpec2, Seq.empty) val analyzed = query.analyze val optimized = Optimize.execute(analyzed) val correctAnswer = testRelation - .window(Seq(sum(c).as('sum_a_1)), partitionSpec2, Seq.empty) - .window(Seq(sum(c).as('sum_a_2)), partitionSpec4, Seq.empty) - .select('a, 'b, 'c, 'd, 'sum_a_2, 'sum_a_1) + .window(Seq(sum(c).as("sum_a_1")), partitionSpec2, Seq.empty) + .window(Seq(sum(c).as("sum_a_2")), partitionSpec4, Seq.empty) + .select("a".attr, "b".attr, "c".attr, "d".attr, "sum_a_2".attr, "sum_a_1".attr) comparePlans(optimized, correctAnswer.analyze) } test("don't transpose two adjacent windows with incompatible partitions") { val query = testRelation - .window(Seq(sum(c).as('sum_a_2)), partitionSpec3, Seq.empty) - .window(Seq(sum(c).as('sum_a_1)), partitionSpec1, Seq.empty) + .window(Seq(sum(c).as("sum_a_2")), partitionSpec3, Seq.empty) + .window(Seq(sum(c).as("sum_a_1")), partitionSpec1, Seq.empty) val analyzed = query.analyze val optimized = Optimize.execute(analyzed) @@ -91,8 +91,8 @@ class TransposeWindowSuite extends PlanTest { test("don't transpose two adjacent windows with intersection of partition and output set") { val query = testRelation - .window(Seq(('a + 'b).as('e), sum(c).as('sum_a_2)), partitionSpec3, Seq.empty) - .window(Seq(sum(c).as('sum_a_1)), Seq(a, 'e), Seq.empty) + .window(Seq(("a".attr + "b".attr).as("e"), sum(c).as("sum_a_2")), partitionSpec3, Seq.empty) + .window(Seq(sum(c).as("sum_a_1")), Seq(a, "e".attr), Seq.empty) val analyzed = query.analyze val optimized = Optimize.execute(analyzed) @@ -102,8 +102,8 @@ class TransposeWindowSuite extends PlanTest { test("don't transpose two adjacent windows with non-deterministic expressions") { val query = testRelation - .window(Seq(Rand(0).as('e), sum(c).as('sum_a_2)), partitionSpec3, Seq.empty) - .window(Seq(sum(c).as('sum_a_1)), partitionSpec1, Seq.empty) + .window(Seq(Rand(0).as("e"), sum(c).as("sum_a_2")), partitionSpec3, Seq.empty) + .window(Seq(sum(c).as("sum_a_1")), partitionSpec1, Seq.empty) val analyzed = query.analyze val optimized = Optimize.execute(analyzed) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala index 5fc99a3a57c0..94085ca6d17b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala @@ -39,7 +39,7 @@ class TypedFilterOptimizationSuite extends PlanTest { implicit private def productEncoder[T <: Product : TypeTag] = ExpressionEncoder[T]() - val testRelation = LocalRelation('_1.int, '_2.int) + val testRelation = LocalRelation("_1".attr.int, "_2".attr.int) test("filter after serialize with the same object type") { val f = (i: (Int, Int)) => i._1 > 0 @@ -53,7 +53,7 @@ class TypedFilterOptimizationSuite extends PlanTest { val expected = testRelation .deserialize[(Int, Int)] - .where(callFunction(f, BooleanType, 'obj)) + .where(callFunction(f, BooleanType, "obj".attr)) .serialize[(Int, Int)].analyze comparePlans(optimized, expected) @@ -82,7 +82,7 @@ class TypedFilterOptimizationSuite extends PlanTest { val expected = testRelation .deserialize[(Int, Int)] - .where(callFunction(f, BooleanType, 'obj)) + .where(callFunction(f, BooleanType, "obj".attr)) .serialize[(Int, Int)].analyze comparePlans(optimized, expected) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnwrapCastInBinaryComparisonSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnwrapCastInBinaryComparisonSuite.scala index 0afb166b80ca..4ce8ceeb0ed9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnwrapCastInBinaryComparisonSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnwrapCastInBinaryComparisonSuite.scala @@ -36,10 +36,11 @@ class UnwrapCastInBinaryComparisonSuite extends PlanTest with ExpressionEvalHelp NullPropagation, UnwrapCastInBinaryComparison) :: Nil } - val testRelation: LocalRelation = LocalRelation('a.short, 'b.float, 'c.decimal(5, 2)) - val f: BoundReference = 'a.short.canBeNull.at(0) - val f2: BoundReference = 'b.float.canBeNull.at(1) - val f3: BoundReference = 'c.decimal(5, 2).canBeNull.at(2) + val testRelation: LocalRelation = + LocalRelation("a".attr.short, "b".attr.float, "c".attr.decimal(5, 2)) + val f: BoundReference = "a".attr.short.canBeNull.at(0) + val f2: BoundReference = "b".attr.float.canBeNull.at(1) + val f3: BoundReference = "c".attr.decimal(5, 2).canBeNull.at(2) test("unwrap casts when literal == max") { val v = Short.MaxValue diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala index dcd2fbbf0052..4dd52dd04b8e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala @@ -48,11 +48,12 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { SimplifyExtractValueOps) :: Nil } - private val idAtt = ('id).long.notNull - private val nullableIdAtt = ('nullable_id).long + private val idAtt = "id".attr.long.notNull + private val nullableIdAtt = "nullable_id".attr.long private val relation = LocalRelation(idAtt, nullableIdAtt) - private val testRelation = LocalRelation('a.int, 'b.int, 'c.int, 'd.double, 'e.int) + private val testRelation = + LocalRelation("a".attr.int, "b".attr.int, "c".attr.int, "d".attr.double, "e".attr.int) private def checkRule(originalQuery: LogicalPlan, correctAnswer: LogicalPlan) = { val optimized = Optimizer.execute(originalQuery.analyze) @@ -64,29 +65,29 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { val query = relation .select( GetStructField( - CreateNamedStruct(Seq("att", 'id )), + CreateNamedStruct(Seq("att", "id".attr)), 0, None) as "outerAtt") - val expected = relation.select('id as "outerAtt") + val expected = relation.select("id".attr as "outerAtt") checkRule(query, expected) } test("explicit get from named_struct- expression maintains original deduced alias") { val query = relation - .select(GetStructField(CreateNamedStruct(Seq("att", 'id)), 0, None)) + .select(GetStructField(CreateNamedStruct(Seq("att", "id".attr)), 0, None)) val expected = relation - .select('id as "named_struct(att, id).att") + .select("id".attr as "named_struct(att, id).att") checkRule(query, expected) } test("collapsed getStructField ontop of namedStruct") { val query = relation - .select(CreateNamedStruct(Seq("att", 'id)) as "struct1") - .select(GetStructField('struct1, 0, None) as "struct1Att") - val expected = relation.select('id as "struct1Att") + .select(CreateNamedStruct(Seq("att", "id".attr)) as "struct1") + .select(GetStructField("struct1".attr, 0, None) as "struct1Att") + val expected = relation.select("id".attr as "struct1Att") checkRule(query, expected) } @@ -94,17 +95,17 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { val query = relation .select( CreateNamedStruct(Seq( - "att1", 'id, - "att2", 'id * 'id)) as "struct1") + "att1", "id".attr, + "att2", "id".attr * "id".attr)) as "struct1") .select( - GetStructField('struct1, 0, None) as "struct1Att1", - GetStructField('struct1, 1, None) as "struct1Att2") + GetStructField("struct1".attr, 0, None) as "struct1Att1", + GetStructField("struct1".attr, 1, None) as "struct1Att2") val expected = relation. select( - 'id as "struct1Att1", - ('id * 'id) as "struct1Att2") + "id".attr as "struct1Att1", + ("id".attr * "id".attr) as "struct1Att2") checkRule(query, expected) } @@ -113,17 +114,17 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { val query = relation .select( CreateNamedStruct(Seq( - "att1", 'id, - "att2", 'id * 'id)) as "struct1") + "att1", "id".attr, + "att2", "id".attr * "id".attr)) as "struct1") .select( - GetStructField('struct1, 0, None), - GetStructField('struct1, 1, None)) + GetStructField("struct1".attr, 0, None), + GetStructField("struct1".attr, 1, None)) val expected = relation. select( - 'id as "struct1.att1", - ('id * 'id) as "struct1.att2") + "id".attr as "struct1.att1", + ("id".attr * "id".attr) as "struct1.att2") checkRule(query, expected) } @@ -132,21 +133,21 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { val rel = relation.select( CreateArray(Seq( CreateNamedStruct(Seq( - "att1", 'id, - "att2", 'id * 'id)), + "att1", "id".attr, + "att2", "id".attr * "id".attr)), CreateNamedStruct(Seq( - "att1", 'id + 1, - "att2", ('id + 1) * ('id + 1)) + "att1", "id".attr + 1, + "att2", ("id".attr + 1) * ("id".attr + 1)) )) ) as "arr" ) val query = rel .select( - GetArrayStructFields('arr, StructField("att1", LongType, false), 0, 1, false) as "a1", - GetArrayItem('arr, 1) as "a2", - GetStructField(GetArrayItem('arr, 1), 0, None) as "a3", + GetArrayStructFields("arr".attr, StructField("att1", LongType, false), 0, 1, false) as "a1", + GetArrayItem("arr".attr, 1) as "a2", + GetStructField(GetArrayItem("arr".attr, 1), 0, None) as "a3", GetArrayItem( - GetArrayStructFields('arr, + GetArrayStructFields("arr".attr, StructField("att1", LongType, false), 0, 1, @@ -155,12 +156,12 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { val expected = relation .select( - CreateArray(Seq('id, 'id + 1L)) as "a1", + CreateArray(Seq("id".attr, "id".attr + 1L)) as "a1", CreateNamedStruct(Seq( - "att1", ('id + 1L), - "att2", (('id + 1L) * ('id + 1L)))) as "a2", - ('id + 1L) as "a3", - ('id + 1L) as "a4") + "att1", ("id".attr + 1L), + "att2", (("id".attr + 1L) * ("id".attr + 1L)))) as "a2", + ("id".attr + 1L) as "a3", + ("id".attr + 1L) as "a4") checkRule(query, expected) } @@ -179,19 +180,19 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { val rel = relation .select( CreateMap(Seq( - "r1", CreateNamedStruct(Seq("att1", 'id)), - "r2", CreateNamedStruct(Seq("att1", ('id + 1L))))) as "m") + "r1", CreateNamedStruct(Seq("att1", "id".attr)), + "r2", CreateNamedStruct(Seq("att1", ("id".attr + 1L))))) as "m") val query = rel .select( - GetMapValue('m, "r1") as "a1", - GetStructField(GetMapValue('m, "r1"), 0, None) as "a2", - GetMapValue('m, "r32") as "a3", - GetStructField(GetMapValue('m, "r32"), 0, None) as "a4") + GetMapValue("m".attr, "r1") as "a1", + GetStructField(GetMapValue("m".attr, "r1"), 0, None) as "a2", + GetMapValue("m".attr, "r32") as "a3", + GetStructField(GetMapValue("m".attr, "r32"), 0, None) as "a4") val expected = relation.select( - CreateNamedStruct(Seq("att1", 'id)) as "a1", - 'id as "a2", + CreateNamedStruct(Seq("att1", "id".attr)) as "a1", + "id".attr as "a2", Literal.create( null, StructType( @@ -206,21 +207,21 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { val query = relation.select( GetMapValue( CreateMap(Seq( - 'id, ('id + 1L), - ('id + 1L), ('id + 2L), - ('id + 2L), ('id + 3L), - Literal(13L), 'id, - ('id + 3L), ('id + 4L), - ('id + 4L), ('id + 5L))), + "id".attr, ("id".attr + 1L), + ("id".attr + 1L), ("id".attr + 2L), + ("id".attr + 2L), ("id".attr + 3L), + Literal(13L), "id".attr, + ("id".attr + 3L), ("id".attr + 4L), + ("id".attr + 4L), ("id".attr + 5L))), 13L) as "a") val expected = relation .select( CaseWhen(Seq( - (EqualTo(13L, 'id), ('id + 1L)), - (EqualTo(13L, ('id + 1L)), ('id + 2L)), - (EqualTo(13L, ('id + 2L)), ('id + 3L)), - (Literal(true), 'id))) as "a") + (EqualTo(13L, "id".attr), ("id".attr + 1L)), + (EqualTo(13L, ("id".attr + 1L)), ("id".attr + 2L)), + (EqualTo(13L, ("id".attr + 2L)), ("id".attr + 3L)), + (Literal(true), "id".attr))) as "a") checkRule(query, expected) } @@ -229,19 +230,19 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { .select( GetMapValue( CreateMap(Seq( - 'id, ('id + 1L), - ('id + 1L), ('id + 2L), - ('id + 2L), ('id + 3L), - ('id + 3L), ('id + 4L), - ('id + 4L), ('id + 5L))), - ('id + 3L)) as "a") + "id".attr, ("id".attr + 1L), + ("id".attr + 1L), ("id".attr + 2L), + ("id".attr + 2L), ("id".attr + 3L), + ("id".attr + 3L), ("id".attr + 4L), + ("id".attr + 4L), ("id".attr + 5L))), + ("id".attr + 3L)) as "a") val expected = relation .select( CaseWhen(Seq( - (EqualTo('id + 3L, 'id), ('id + 1L)), - (EqualTo('id + 3L, ('id + 1L)), ('id + 2L)), - (EqualTo('id + 3L, ('id + 2L)), ('id + 3L)), - (Literal(true), ('id + 4L)))) as "a") + (EqualTo("id".attr + 3L, "id".attr), ("id".attr + 1L)), + (EqualTo("id".attr + 3L, ("id".attr + 1L)), ("id".attr + 2L)), + (EqualTo("id".attr + 3L, ("id".attr + 2L)), ("id".attr + 3L)), + (Literal(true), ("id".attr + 4L)))) as "a") checkRule(query, expected) } @@ -250,19 +251,19 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { .select( GetMapValue( CreateMap(Seq( - 'id, ('id + 1L), - ('id + 1L), ('id + 2L), - ('id + 2L), ('id + 3L), - ('id + 3L), ('id + 4L), - ('id + 4L), ('id + 5L))), - 'id + 30L) as "a") + "id".attr, ("id".attr + 1L), + ("id".attr + 1L), ("id".attr + 2L), + ("id".attr + 2L), ("id".attr + 3L), + ("id".attr + 3L), ("id".attr + 4L), + ("id".attr + 4L), ("id".attr + 5L))), + "id".attr + 30L) as "a") val expected = relation.select( CaseWhen(Seq( - (EqualTo('id + 30L, 'id), ('id + 1L)), - (EqualTo('id + 30L, ('id + 1L)), ('id + 2L)), - (EqualTo('id + 30L, ('id + 2L)), ('id + 3L)), - (EqualTo('id + 30L, ('id + 3L)), ('id + 4L)), - (EqualTo('id + 30L, ('id + 4L)), ('id + 5L)))) as "a") + (EqualTo("id".attr + 30L, "id".attr), ("id".attr + 1L)), + (EqualTo("id".attr + 30L, ("id".attr + 1L)), ("id".attr + 2L)), + (EqualTo("id".attr + 30L, ("id".attr + 2L)), ("id".attr + 3L)), + (EqualTo("id".attr + 30L, ("id".attr + 3L)), ("id".attr + 4L)), + (EqualTo("id".attr + 30L, ("id".attr + 4L)), ("id".attr + 5L)))) as "a") checkRule(rel, expected) } @@ -271,22 +272,22 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { .select( GetMapValue( CreateMap(Seq( - 'id, ('id + 1L), - ('id + 1L), ('id + 2L), - ('id + 2L), ('id + 3L), - Literal(14L), 'id, - ('id + 3L), ('id + 4L), - ('id + 4L), ('id + 5L))), + "id".attr, ("id".attr + 1L), + ("id".attr + 1L), ("id".attr + 2L), + ("id".attr + 2L), ("id".attr + 3L), + Literal(14L), "id".attr, + ("id".attr + 3L), ("id".attr + 4L), + ("id".attr + 4L), ("id".attr + 5L))), 13L) as "a") val expected = relation .select( CaseKeyWhen(13L, - Seq('id, ('id + 1L), - ('id + 1L), ('id + 2L), - ('id + 2L), ('id + 3L), - ('id + 3L), ('id + 4L), - ('id + 4L), ('id + 5L))) as "a") + Seq("id".attr, ("id".attr + 1L), + ("id".attr + 1L), ("id".attr + 2L), + ("id".attr + 2L), ("id".attr + 3L), + ("id".attr + 3L), ("id".attr + 4L), + ("id".attr + 4L), ("id".attr + 5L))) as "a") checkRule(rel, expected) } @@ -296,100 +297,101 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { .select( GetMapValue( CreateMap(Seq( - 'id, ('id + 1L), - ('id + 1L), ('id + 2L), - ('id + 2L), Literal.create(null, LongType), - Literal(2L), 'id, - ('id + 3L), ('id + 4L), - ('id + 4L), ('id + 5L))), + "id".attr, ("id".attr + 1L), + ("id".attr + 1L), ("id".attr + 2L), + ("id".attr + 2L), Literal.create(null, LongType), + Literal(2L), "id".attr, + ("id".attr + 3L), ("id".attr + 4L), + ("id".attr + 4L), ("id".attr + 5L))), 2L ) as "a") val expected = relation .select( CaseWhen(Seq( - (EqualTo(2L, 'id), ('id + 1L)), + (EqualTo(2L, "id".attr), ("id".attr + 1L)), // these two are possible matches, we can't tell until runtime - (EqualTo(2L, ('id + 1L)), ('id + 2L)), - (EqualTo(2L, 'id + 2L), Literal.create(null, LongType)), + (EqualTo(2L, ("id".attr + 1L)), ("id".attr + 2L)), + (EqualTo(2L, "id".attr + 2L), Literal.create(null, LongType)), // this is a definite match (two constants), - // but it cannot override a potential match with ('id + 2L), + // but it cannot override a potential match with ("id".attr + 2L), // which is exactly what [[Coalesce]] would do in this case. - (Literal.TrueLiteral, 'id))) as "a") + (Literal.TrueLiteral, "id".attr))) as "a") checkRule(rel, expected) } test("SPARK-23500: Simplify array ops that are not at the top node") { - val query = LocalRelation('id.long) + val query = LocalRelation("id".attr.long) .select( CreateArray(Seq( CreateNamedStruct(Seq( - "att1", 'id, - "att2", 'id * 'id)), + "att1", "id".attr, + "att2", "id".attr * "id".attr)), CreateNamedStruct(Seq( - "att1", 'id + 1, - "att2", ('id + 1) * ('id + 1)) + "att1", "id".attr + 1, + "att2", ("id".attr + 1) * ("id".attr + 1)) )) ) as "arr") .select( - GetStructField(GetArrayItem('arr, 1), 0, None) as "a1", + GetStructField(GetArrayItem("arr".attr, 1), 0, None) as "a1", GetArrayItem( - GetArrayStructFields('arr, + GetArrayStructFields("arr".attr, StructField("att1", LongType, nullable = false), ordinal = 0, numFields = 1, containsNull = false), ordinal = 1) as "a2") - .orderBy('id.asc) + .orderBy("id".attr.asc) - val expected = LocalRelation('id.long) + val expected = LocalRelation("id".attr.long) .select( - ('id + 1L) as "a1", - ('id + 1L) as "a2") - .orderBy('id.asc) + ("id".attr + 1L) as "a1", + ("id".attr + 1L) as "a2") + .orderBy("id".attr.asc) checkRule(query, expected) } test("SPARK-23500: Simplify map ops that are not top nodes") { val query = - LocalRelation('id.long) + LocalRelation("id".attr.long) .select( CreateMap(Seq( - "r1", 'id, - "r2", 'id + 1L)) as "m") + "r1", "id".attr, + "r2", "id".attr + 1L)) as "m") .select( - GetMapValue('m, "r1") as "a1", - GetMapValue('m, "r32") as "a2") - .orderBy('id.asc) - .select('a1, 'a2) + GetMapValue("m".attr, "r1") as "a1", + GetMapValue("m".attr, "r32") as "a2") + .orderBy("id".attr.asc) + .select("a1".attr, "a2".attr) val expected = - LocalRelation('id.long).select( - 'id as "a1", + LocalRelation("id".attr.long).select( + "id".attr as "a1", Literal.create(null, LongType) as "a2") - .orderBy('id.asc) + .orderBy("id".attr.asc) checkRule(query, expected) } test("SPARK-23500: Simplify complex ops that aren't at the plan root") { val structRel = relation - .select(GetStructField(CreateNamedStruct(Seq("att1", 'nullable_id)), 0, None) as "foo") + .select(GetStructField(CreateNamedStruct(Seq("att1", "nullable_id".attr)), 0, None) as "foo") .groupBy($"foo")("1") val structExpected = relation - .select('nullable_id as "foo") + .select("nullable_id".attr as "foo") .groupBy($"foo")("1") checkRule(structRel, structExpected) val arrayRel = relation - .select(GetArrayItem(CreateArray(Seq('nullable_id, 'nullable_id + 1L)), 0) as "a1") + .select(GetArrayItem( + CreateArray(Seq("nullable_id".attr, "nullable_id".attr + 1L)), 0) as "a1") .groupBy($"a1")("1") - val arrayExpected = relation.select('nullable_id as "a1").groupBy($"a1")("1") + val arrayExpected = relation.select("nullable_id".attr as "a1").groupBy($"a1")("1") checkRule(arrayRel, arrayExpected) val mapRel = relation - .select(GetMapValue(CreateMap(Seq("id", 'nullable_id)), "id") as "m1") + .select(GetMapValue(CreateMap(Seq("id", "nullable_id".attr)), "id") as "m1") .groupBy($"m1")("1") val mapExpected = relation - .select('nullable_id as "m1") + .select("nullable_id".attr as "m1") .groupBy($"m1")("1") checkRule(mapRel, mapExpected) } @@ -398,19 +400,19 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { // Make sure that aggregation exprs are correctly ignored. Maps can't be used in // grouping exprs so aren't tested here. val structAggRel = relation.groupBy( - CreateNamedStruct(Seq("att1", 'nullable_id)))( - GetStructField(CreateNamedStruct(Seq("att1", 'nullable_id)), 0, None)) + CreateNamedStruct(Seq("att1", "nullable_id".attr)))( + GetStructField(CreateNamedStruct(Seq("att1", "nullable_id".attr)), 0, None)) checkRule(structAggRel, structAggRel) val arrayAggRel = relation.groupBy( - CreateArray(Seq('nullable_id)))(GetArrayItem(CreateArray(Seq('nullable_id)), 0)) + CreateArray(Seq("nullable_id".attr)))(GetArrayItem(CreateArray(Seq("nullable_id".attr)), 0)) checkRule(arrayAggRel, arrayAggRel) // This could be done if we had a more complex rule that checks that // the CreateMap does not come from key. val originalQuery = relation - .groupBy('id)( - GetMapValue(CreateMap(Seq('id, 'id + 1L)), 0L) as "a" + .groupBy("id".attr)( + GetMapValue(CreateMap(Seq("id".attr, "id".attr + 1L)), 0L) as "a" ) checkRule(originalQuery, originalQuery) } @@ -419,13 +421,13 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { val originalQuery = testRelation .select( - namedStruct("col1", 'b, "col2", 'c).as("s1"), 'a, 'b) - .select('s1 getField "col2" as 's1Col2, - namedStruct("col1", 'a, "col2", 'b).as("s2")) - .select('s1Col2, 's2 getField "col2" as 's2Col2) + namedStruct("col1", "b".attr, "col2", "c".attr).as("s1"), "a".attr, "b".attr) + .select("s1".attr getField "col2" as "s1Col2", + namedStruct("col1", "a".attr, "col2", "b".attr).as("s2")) + .select("s1Col2".attr, "s2".attr getField "col2" as "s2Col2") val correctAnswer = testRelation - .select('c as 's1Col2, 'b as 's2Col2) + .select("c".attr as "s1Col2", "b".attr as "s2Col2") checkRule(originalQuery, correctAnswer) } @@ -433,11 +435,11 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { val originalQuery = testRelation .select( - namedStruct("col1", 'b, "col2", 'c) getField "col2" as 'sCol2, - namedStruct("col1", 'a, "col2", 'c) getField "col1" as 'sCol1) + namedStruct("col1", "b".attr, "col2", "c".attr) getField "col2" as "sCol2", + namedStruct("col1", "a".attr, "col2", "c".attr) getField "col1" as "sCol1") val correctAnswer = testRelation - .select('c as 'sCol2, 'a as 'sCol1) + .select("c".attr as "sCol2", "a".attr as "sCol1") checkRule(originalQuery, correctAnswer) } @@ -454,16 +456,16 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { checkEvaluation(GetMapValue(mb0, Literal(Array[Byte](3, 4))), null) } - private val structAttr = 'struct1.struct('a.int, 'b.int).withNullability(false) + private val structAttr = "struct1".attr.struct("a".attr.int, "b".attr.int).withNullability(false) private val testStructRelation = LocalRelation(structAttr) - private val nullableStructAttr = 'struct1.struct('a.int, 'b.int) + private val nullableStructAttr = "struct1".attr.struct("a".attr.int, "b".attr.int) private val testNullableStructRelation = LocalRelation(nullableStructAttr) test("simplify GetStructField on basic UpdateFields") { def check(fieldOps: Seq[StructFieldsOperation], ordinal: Int, expected: Expression): Unit = { def query(relation: LocalRelation): LogicalPlan = - relation.select(GetStructField(UpdateFields('struct1, fieldOps), ordinal).as("res")) + relation.select(GetStructField(UpdateFields("struct1".attr, fieldOps), ordinal).as("res")) checkRule( query(testStructRelation), @@ -473,30 +475,30 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { query(testNullableStructRelation), testNullableStructRelation.select((expected match { case expr: GetStructField => expr - case expr => If(IsNull('struct1), Literal(null, expr.dataType), expr) + case expr => If(IsNull("struct1".attr), Literal(null, expr.dataType), expr) }).as("res"))) } // scalastyle:off line.size.limit // add attribute, extract an attribute from the original struct - check(WithField("c", Literal(3)) :: Nil, 0, GetStructField('struct1, 0)) - check(WithField("c", Literal(3)) :: Nil, 1, GetStructField('struct1, 1)) + check(WithField("c", Literal(3)) :: Nil, 0, GetStructField("struct1".attr, 0)) + check(WithField("c", Literal(3)) :: Nil, 1, GetStructField("struct1".attr, 1)) // add attribute, extract added attribute check(WithField("c", Literal(3)) :: Nil, 2, Literal(3)) // replace attribute, extract an attribute from the original struct - check(WithField("a", Literal(1)) :: Nil, 1, GetStructField('struct1, 1)) - check(WithField("b", Literal(2)) :: Nil, 0, GetStructField('struct1, 0)) + check(WithField("a", Literal(1)) :: Nil, 1, GetStructField("struct1".attr, 1)) + check(WithField("b", Literal(2)) :: Nil, 0, GetStructField("struct1".attr, 0)) // replace attribute, extract replaced attribute check(WithField("a", Literal(1)) :: Nil, 0, Literal(1)) check(WithField("b", Literal(2)) :: Nil, 1, Literal(2)) // add multiple attributes, extract an attribute from the original struct - check(WithField("c", Literal(3)) :: WithField("c", Literal(4)) :: Nil, 0, GetStructField('struct1, 0)) - check(WithField("c", Literal(3)) :: WithField("d", Literal(4)) :: Nil, 0, GetStructField('struct1, 0)) - check(WithField("c", Literal(3)) :: WithField("c", Literal(4)) :: Nil, 1, GetStructField('struct1, 1)) - check(WithField("c", Literal(3)) :: WithField("d", Literal(4)) :: Nil, 1, GetStructField('struct1, 1)) + check(WithField("c", Literal(3)) :: WithField("c", Literal(4)) :: Nil, 0, GetStructField("struct1".attr, 0)) + check(WithField("c", Literal(3)) :: WithField("d", Literal(4)) :: Nil, 0, GetStructField("struct1".attr, 0)) + check(WithField("c", Literal(3)) :: WithField("c", Literal(4)) :: Nil, 1, GetStructField("struct1".attr, 1)) + check(WithField("c", Literal(3)) :: WithField("d", Literal(4)) :: Nil, 1, GetStructField("struct1".attr, 1)) // add multiple attributes, extract newly added attribute check(WithField("c", Literal(3)) :: WithField("c", Literal(4)) :: Nil, 2, Literal(4)) check(WithField("c", Literal(4)) :: WithField("c", Literal(3)) :: Nil, 2, Literal(3)) @@ -506,45 +508,45 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { check(WithField("d", Literal(4)) :: WithField("c", Literal(3)) :: Nil, 3, Literal(3)) // drop attribute, extract an attribute from the original struct - check(DropField("b") :: Nil, 0, GetStructField('struct1, 0)) - check(DropField("a") :: Nil, 0, GetStructField('struct1, 1)) + check(DropField("b") :: Nil, 0, GetStructField("struct1".attr, 0)) + check(DropField("a") :: Nil, 0, GetStructField("struct1".attr, 1)) // drop attribute, add attribute, extract an attribute from the original struct - check(DropField("b") :: WithField("c", Literal(3)) :: Nil, 0, GetStructField('struct1, 0)) - check(DropField("a") :: WithField("c", Literal(3)) :: Nil, 0, GetStructField('struct1, 1)) + check(DropField("b") :: WithField("c", Literal(3)) :: Nil, 0, GetStructField("struct1".attr, 0)) + check(DropField("a") :: WithField("c", Literal(3)) :: Nil, 0, GetStructField("struct1".attr, 1)) // drop attribute, add attribute, extract added attribute check(DropField("b") :: WithField("c", Literal(3)) :: Nil, 1, Literal(3)) check(DropField("a") :: WithField("c", Literal(3)) :: Nil, 1, Literal(3)) // add attribute, drop attribute, extract an attribute from the original struct - check(WithField("c", Literal(3)) :: DropField("a") :: Nil, 0, GetStructField('struct1, 1)) - check(WithField("c", Literal(3)) :: DropField("b") :: Nil, 0, GetStructField('struct1, 0)) + check(WithField("c", Literal(3)) :: DropField("a") :: Nil, 0, GetStructField("struct1".attr, 1)) + check(WithField("c", Literal(3)) :: DropField("b") :: Nil, 0, GetStructField("struct1".attr, 0)) // add attribute, drop attribute, extract added attribute check(WithField("c", Literal(3)) :: DropField("a") :: Nil, 1, Literal(3)) check(WithField("c", Literal(3)) :: DropField("b") :: Nil, 1, Literal(3)) // replace attribute, drop same attribute, extract an attribute from the original struct - check(WithField("b", Literal(3)) :: DropField("b") :: Nil, 0, GetStructField('struct1, 0)) - check(WithField("a", Literal(3)) :: DropField("a") :: Nil, 0, GetStructField('struct1, 1)) + check(WithField("b", Literal(3)) :: DropField("b") :: Nil, 0, GetStructField("struct1".attr, 0)) + check(WithField("a", Literal(3)) :: DropField("a") :: Nil, 0, GetStructField("struct1".attr, 1)) // add attribute, drop same attribute, extract an attribute from the original struct - check(WithField("c", Literal(3)) :: DropField("c") :: Nil, 0, GetStructField('struct1, 0)) - check(WithField("c", Literal(3)) :: DropField("c") :: Nil, 1, GetStructField('struct1, 1)) + check(WithField("c", Literal(3)) :: DropField("c") :: Nil, 0, GetStructField("struct1".attr, 0)) + check(WithField("c", Literal(3)) :: DropField("c") :: Nil, 1, GetStructField("struct1".attr, 1)) // replace attribute, drop another attribute, extract added attribute check(WithField("b", Literal(3)) :: DropField("a") :: Nil, 0, Literal(3)) check(WithField("a", Literal(3)) :: DropField("b") :: Nil, 0, Literal(3)) // drop attribute, add same attribute, extract attribute from the original struct - check(DropField("b") :: WithField("b", Literal(3)) :: Nil, 0, GetStructField('struct1, 0)) - check(DropField("a") :: WithField("a", Literal(3)) :: Nil, 0, GetStructField('struct1, 1)) + check(DropField("b") :: WithField("b", Literal(3)) :: Nil, 0, GetStructField("struct1".attr, 0)) + check(DropField("a") :: WithField("a", Literal(3)) :: Nil, 0, GetStructField("struct1".attr, 1)) // drop attribute, add same attribute, extract added attribute check(DropField("b") :: WithField("b", Literal(3)) :: Nil, 1, Literal(3)) check(DropField("a") :: WithField("a", Literal(3)) :: Nil, 1, Literal(3)) // drop non-existent attribute, add same attribute, extract attribute from the original struct - check(DropField("c") :: WithField("c", Literal(3)) :: Nil, 0, GetStructField('struct1, 0)) - check(DropField("c") :: WithField("c", Literal(3)) :: Nil, 1, GetStructField('struct1, 1)) + check(DropField("c") :: WithField("c", Literal(3)) :: Nil, 0, GetStructField("struct1".attr, 0)) + check(DropField("c") :: WithField("c", Literal(3)) :: Nil, 1, GetStructField("struct1".attr, 1)) // drop non-existent attribute, add same attribute, extract added attribute check(DropField("c") :: WithField("c", Literal(3)) :: Nil, 2, Literal(3)) @@ -552,7 +554,7 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { } test("simplify GetStructField that is extracting a field nested inside a struct") { - val struct2 = 'struct2.struct('b.int) + val struct2 = "struct2".attr.struct("b".attr.int) val testStructRelation = LocalRelation(structAttr, struct2) val testNullableStructRelation = LocalRelation(nullableStructAttr, struct2) @@ -561,15 +563,16 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { def addFieldFromSameStructAndThenExtractIt(relation: LocalRelation): LogicalPlan = relation.select(GetStructField( - UpdateFields('struct1, WithField("b", GetStructField('struct1, 0)) :: Nil), 1).as("res")) + UpdateFields("struct1".attr, + WithField("b", GetStructField("struct1".attr, 0)) :: Nil), 1).as("res")) checkRule( addFieldFromSameStructAndThenExtractIt(testStructRelation), - testStructRelation.select(GetStructField('struct1, 0).as("res"))) + testStructRelation.select(GetStructField("struct1".attr, 0).as("res"))) checkRule( addFieldFromSameStructAndThenExtractIt(testNullableStructRelation), - testNullableStructRelation.select(GetStructField('struct1, 0).as("res"))) + testNullableStructRelation.select(GetStructField("struct1".attr, 0).as("res"))) // if the field being extracted is from a different struct than the one UpdateFields is // modifying, we must return GetStructField wrapped in If(IsNull(struct), null, GetStructField) @@ -577,16 +580,18 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { def addFieldFromAnotherStructAndThenExtractIt(relation: LocalRelation): LogicalPlan = relation.select(GetStructField( - UpdateFields('struct1, WithField("b", GetStructField('struct2, 0)) :: Nil), 1).as("res")) + UpdateFields("struct1".attr, + WithField("b", GetStructField("struct2".attr, 0)) :: Nil), 1).as("res")) checkRule( addFieldFromAnotherStructAndThenExtractIt(testStructRelation), - testStructRelation.select(GetStructField('struct2, 0).as("res"))) + testStructRelation.select(GetStructField("struct2".attr, 0).as("res"))) checkRule( addFieldFromAnotherStructAndThenExtractIt(testNullableStructRelation), testNullableStructRelation.select( - If(IsNull('struct1), Literal(null, IntegerType), GetStructField('struct2, 0)).as("res"))) + If(IsNull("struct1".attr), + Literal(null, IntegerType), GetStructField("struct2".attr, 0)).as("res"))) } test("simplify GetStructField on nested UpdateFields") { @@ -596,7 +601,7 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { UpdateFields( UpdateFields( UpdateFields( - 'struct1, + "struct1".attr, WithField("c", Literal(1)) :: Nil), WithField("d", Literal(2)) :: Nil), WithField("e", Literal(3)) :: Nil), @@ -614,79 +619,79 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { checkRule( query(testNullableStructRelation, 5), testNullableStructRelation.select( - If(IsNull('struct1), Literal(null, IntegerType), Literal(4)) as "res")) + If(IsNull("struct1".attr), Literal(null, IntegerType), Literal(4)) as "res")) // extract field from original struct checkRule( query(testStructRelation, 0), - testStructRelation.select(GetStructField('struct1, 0) as "res")) + testStructRelation.select(GetStructField("struct1".attr, 0) as "res")) checkRule( query(testNullableStructRelation, 0), - testNullableStructRelation.select(GetStructField('struct1, 0) as "res")) + testNullableStructRelation.select(GetStructField("struct1".attr, 0) as "res")) } test("simplify multiple GetStructField on the same UpdateFields") { def query(relation: LocalRelation): LogicalPlan = relation - .select(UpdateFields('struct1, WithField("b", Literal(2)) :: Nil) as "struct2") + .select(UpdateFields("struct1".attr, WithField("b", Literal(2)) :: Nil) as "struct2") .select( - GetStructField('struct2, 0, Some("a")) as "struct1A", - GetStructField('struct2, 1, Some("b")) as "struct1B") + GetStructField("struct2".attr, 0, Some("a")) as "struct1A", + GetStructField("struct2".attr, 1, Some("b")) as "struct1B") checkRule( query(testStructRelation), testStructRelation.select( - GetStructField('struct1, 0) as "struct1A", + GetStructField("struct1".attr, 0) as "struct1A", Literal(2) as "struct1B")) checkRule( query(testNullableStructRelation), testNullableStructRelation.select( - GetStructField('struct1, 0) as "struct1A", - If(IsNull('struct1), Literal(null, IntegerType), Literal(2)) as "struct1B")) + GetStructField("struct1".attr, 0) as "struct1A", + If(IsNull("struct1".attr), Literal(null, IntegerType), Literal(2)) as "struct1B")) } test("simplify multiple GetStructField on different UpdateFields") { def query(relation: LocalRelation): LogicalPlan = relation .select( - UpdateFields('struct1, WithField("b", Literal(2)) :: Nil) as "struct2", - UpdateFields('struct1, WithField("b", Literal(3)) :: Nil) as "struct3") + UpdateFields("struct1".attr, WithField("b", Literal(2)) :: Nil) as "struct2", + UpdateFields("struct1".attr, WithField("b", Literal(3)) :: Nil) as "struct3") .select( - GetStructField('struct2, 0, Some("a")) as "struct2A", - GetStructField('struct2, 1, Some("b")) as "struct2B", - GetStructField('struct3, 0, Some("a")) as "struct3A", - GetStructField('struct3, 1, Some("b")) as "struct3B") + GetStructField("struct2".attr, 0, Some("a")) as "struct2A", + GetStructField("struct2".attr, 1, Some("b")) as "struct2B", + GetStructField("struct3".attr, 0, Some("a")) as "struct3A", + GetStructField("struct3".attr, 1, Some("b")) as "struct3B") checkRule( query(testStructRelation), testStructRelation .select( - GetStructField('struct1, 0) as "struct2A", + GetStructField("struct1".attr, 0) as "struct2A", Literal(2) as "struct2B", - GetStructField('struct1, 0) as "struct3A", + GetStructField("struct1".attr, 0) as "struct3A", Literal(3) as "struct3B")) checkRule( query(testNullableStructRelation), testNullableStructRelation .select( - GetStructField('struct1, 0) as "struct2A", - If(IsNull('struct1), Literal(null, IntegerType), Literal(2)) as "struct2B", - GetStructField('struct1, 0) as "struct3A", - If(IsNull('struct1), Literal(null, IntegerType), Literal(3)) as "struct3B")) + GetStructField("struct1".attr, 0) as "struct2A", + If(IsNull("struct1".attr), Literal(null, IntegerType), Literal(2)) as "struct2B", + GetStructField("struct1".attr, 0) as "struct3A", + If(IsNull("struct1".attr), Literal(null, IntegerType), Literal(3)) as "struct3B")) } test("simplify add multiple nested fields to non-nullable struct") { // this scenario is possible if users add multiple nested columns to a non-nullable struct // using the Column.withField API in a non-performant way val structLevel2 = LocalRelation( - 'a1.struct( - 'a2.struct('a3.int.notNull)).notNull) + "a1".attr.struct( + "a2".attr.struct("a3".attr.int.notNull)).notNull) val query = { - val addB3toA1A2 = UpdateFields('a1, Seq(WithField("a2", - UpdateFields(GetStructField('a1, 0), Seq(WithField("b3", Literal(2))))))) + val addB3toA1A2 = UpdateFields("a1".attr, Seq(WithField("a2", + UpdateFields(GetStructField("a1".attr, 0), Seq(WithField("b3", Literal(2))))))) structLevel2.select( UpdateFields( @@ -696,9 +701,9 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { } val expected = structLevel2.select( - UpdateFields('a1, Seq( + UpdateFields("a1".attr, Seq( // scalastyle:off line.size.limit - WithField("a2", UpdateFields(GetStructField('a1, 0), WithField("b3", 2) :: WithField("c3", 3) :: Nil)) + WithField("a2", UpdateFields(GetStructField("a1".attr, 0), WithField("b3", 2) :: WithField("c3", 3) :: Nil)) // scalastyle:on line.size.limit )).as("a1")) @@ -709,12 +714,12 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { // this scenario is possible if users add multiple nested columns to a nullable struct // using the Column.withField API in a non-performant way val structLevel2 = LocalRelation( - 'a1.struct( - 'a2.struct('a3.int.notNull))) + "a1".attr.struct( + "a2".attr.struct("a3".attr.int.notNull))) val query = { - val addB3toA1A2 = UpdateFields('a1, Seq(WithField("a2", - UpdateFields(GetStructField('a1, 0), Seq(WithField("b3", Literal(2))))))) + val addB3toA1A2 = UpdateFields("a1".attr, Seq(WithField("a2", + UpdateFields(GetStructField("a1".attr, 0), Seq(WithField("b3", Literal(2))))))) structLevel2.select( UpdateFields( @@ -724,15 +729,16 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { } val expected = { - val repeatedExpr = UpdateFields(GetStructField('a1, 0), WithField("b3", Literal(2)) :: Nil) + val repeatedExpr = + UpdateFields(GetStructField("a1".attr, 0), WithField("b3", Literal(2)) :: Nil) val repeatedExprDataType = StructType(Seq( StructField("a3", IntegerType, nullable = false), StructField("b3", IntegerType, nullable = false))) structLevel2.select( - UpdateFields('a1, Seq( + UpdateFields("a1".attr, Seq( WithField("a2", UpdateFields( - If(IsNull('a1), Literal(null, repeatedExprDataType), repeatedExpr), + If(IsNull("a1".attr), Literal(null, repeatedExprDataType), repeatedExpr), WithField("c3", Literal(3)) :: Nil)) )).as("a1")) } @@ -744,13 +750,14 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { // this scenario is possible if users drop multiple nested columns in a non-nullable struct // using the Column.dropFields API in a non-performant way val structLevel2 = LocalRelation( - 'a1.struct( - 'a2.struct('a3.int.notNull, 'b3.int.notNull, 'c3.int.notNull).notNull + "a1".attr.struct( + "a2".attr.struct( + "a3".attr.int.notNull, "b3".attr.int.notNull, "c3".attr.int.notNull).notNull ).notNull) val query = { - val dropA1A2B = UpdateFields('a1, Seq(WithField("a2", UpdateFields( - GetStructField('a1, 0), Seq(DropField("b3")))))) + val dropA1A2B = UpdateFields("a1".attr, Seq(WithField("a2", UpdateFields( + GetStructField("a1".attr, 0), Seq(DropField("b3")))))) structLevel2.select( UpdateFields( @@ -760,8 +767,9 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { } val expected = structLevel2.select( - UpdateFields('a1, Seq( - WithField("a2", UpdateFields(GetStructField('a1, 0), Seq(DropField("b3"), DropField("c3")))) + UpdateFields("a1".attr, Seq( + WithField("a2", + UpdateFields(GetStructField("a1".attr, 0), Seq(DropField("b3"), DropField("c3")))) )).as("a1")) checkRule(query, expected) @@ -771,13 +779,13 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { // this scenario is possible if users drop multiple nested columns in a nullable struct // using the Column.dropFields API in a non-performant way val structLevel2 = LocalRelation( - 'a1.struct( - 'a2.struct('a3.int.notNull, 'b3.int.notNull, 'c3.int.notNull) + "a1".attr.struct( + "a2".attr.struct("a3".attr.int.notNull, "b3".attr.int.notNull, "c3".attr.int.notNull) )) val query = { - val dropA1A2B = UpdateFields('a1, Seq(WithField("a2", UpdateFields( - GetStructField('a1, 0), Seq(DropField("b3")))))) + val dropA1A2B = UpdateFields("a1".attr, Seq(WithField("a2", UpdateFields( + GetStructField("a1".attr, 0), Seq(DropField("b3")))))) structLevel2.select( UpdateFields( @@ -787,15 +795,15 @@ class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { } val expected = { - val repeatedExpr = UpdateFields(GetStructField('a1, 0), DropField("b3") :: Nil) + val repeatedExpr = UpdateFields(GetStructField("a1".attr, 0), DropField("b3") :: Nil) val repeatedExprDataType = StructType(Seq( StructField("a3", IntegerType, nullable = false), StructField("c3", IntegerType, nullable = false))) structLevel2.select( - UpdateFields('a1, Seq( + UpdateFields("a1".attr, Seq( WithField("a2", UpdateFields( - If(IsNull('a1), Literal(null, repeatedExprDataType), repeatedExpr), + If(IsNull("a1".attr), Literal(null, repeatedExprDataType), repeatedExpr), DropField("c3") :: Nil)) )).as("a1")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala index 2e1cf4a137e2..6fc5e8a7d403 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala @@ -230,15 +230,15 @@ class JoinReorderSuite extends JoinReorderPlanTestBase with StatsEstimationTestB test("SPARK-26352: join reordering should not change the order of attributes") { // This test case does not rely on CBO. // It's similar to the test case above, but catches a reordering bug that the one above doesn't - val tab1 = LocalRelation('x.int, 'y.int) - val tab2 = LocalRelation('i.int, 'j.int) - val tab3 = LocalRelation('a.int, 'b.int) + val tab1 = LocalRelation("x".attr.int, "y".attr.int) + val tab2 = LocalRelation("i".attr.int, "j".attr.int) + val tab3 = LocalRelation("a".attr.int, "b".attr.int) val original = tab1.join(tab2, Cross) - .join(tab3, Inner, Some('a === 'x && 'b === 'i)) + .join(tab3, Inner, Some("a".attr === "x".attr && "b".attr === "i".attr)) val expected = - tab1.join(tab3, Inner, Some('a === 'x)) - .join(tab2, Cross, Some('b === 'i)) + tab1.join(tab3, Inner, Some("a".attr === "x".attr)) + .join(tab2, Cross, Some("b".attr === "i".attr)) .select(outputsOf(tab1, tab2, tab3): _*) assertEqualJoinPlans(Optimize, original, expected) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinReorderSuite.scala index ebc12b1d82cf..668a72996f1c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinReorderSuite.scala @@ -141,7 +141,8 @@ class StarJoinReorderSuite extends JoinReorderPlanTestBase with StatsEstimationT size = Some(17), attributeStats = AttributeMap(Seq("s3_pk1", "s3_c2", "s3_c3", "s3_c4").map(nameToColInfo))) - private val d3_ns = LocalRelation('d3_fk1.int, 'd3_c2.int, 'd3_pk1.int, 'd3_c4.int) + private val d3_ns = + LocalRelation("d3_fk1".attr.int, "d3_c2".attr.int, "d3_pk1".attr.int, "d3_c4".attr.int) private val f11 = StatsTestPlan( outputList = Seq("f11_fk1", "f11_fk2", "f11_fk3", "f11_c4").map(nameToAttr), @@ -150,7 +151,7 @@ class StarJoinReorderSuite extends JoinReorderPlanTestBase with StatsEstimationT attributeStats = AttributeMap(Seq("f11_fk1", "f11_fk2", "f11_fk3", "f11_c4") .map(nameToColInfo))) - private val subq = d3.select(sum('d3_fk1).as('col)) + private val subq = d3.select(sum("d3_fk1".attr).as("col")) test("Test 1: Selective star-join on all dimensions") { // Star join: @@ -362,7 +363,7 @@ class StarJoinReorderSuite extends JoinReorderPlanTestBase with StatsEstimationT (nameToAttr("f1_fk3") === "col".attr)) val expected = - d3.select('d3_fk1).select(sum('d3_fk1).as('col)) + d3.select("d3_fk1".attr).select(sum("d3_fk1".attr).as("col")) .join(f1, Inner, Some(nameToAttr("f1_fk3") === "col".attr)) .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk1"))) .join(d2.where(nameToAttr("d2_c2") === 2), Inner, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala index 99051d692451..c3d995c329a1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala @@ -181,9 +181,9 @@ class ErrorParserSuite extends AnalysisTest { |ORDER BY c """.stripMargin, table("t") - .where('a - 'b > 10) - .groupBy('fake - 'breaker)('a, 'b) - .orderBy('c.asc)) + .where("a".attr - "b".attr > 10) + .groupBy("fake".attr - "breaker".attr)("a".attr, "b".attr) + .orderBy("c".attr.asc)) intercept( """ |SELECT * FROM tab diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 0b304a799cdc..6fc4ab26de51 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -76,41 +76,43 @@ class ExpressionParserSuite extends AnalysisTest { // NamedExpression (Alias/Multialias) test("named expressions") { // No Alias - val r0 = 'a + val r0 = "a".attr assertEqual("a", r0) // Single Alias. - val r1 = 'a as "b" + val r1 = "a".attr as "b" assertEqual("a as b", r1) assertEqual("a b", r1) // Multi-Alias - assertEqual("a as (b, c)", MultiAlias('a, Seq("b", "c"))) - assertEqual("a() (b, c)", MultiAlias('a.function(), Seq("b", "c"))) + assertEqual("a as (b, c)", MultiAlias("a".attr, Seq("b", "c"))) + assertEqual("a() (b, c)", MultiAlias("a".attr.function(), Seq("b", "c"))) // Numeric literals without a space between the literal qualifier and the alias, should not be // interpreted as such. An unresolved reference should be returned instead. // TODO add the JIRA-ticket number. - assertEqual("1SL", Symbol("1SL")) + assertEqual("1SL", "1SL".attr) // Aliased star is allowed. - assertEqual("a.* b", UnresolvedStar(Option(Seq("a"))) as 'b) + assertEqual("a.* b", UnresolvedStar(Option(Seq("a"))) as "b") } test("binary logical expressions") { // And - assertEqual("a and b", 'a && 'b) + assertEqual("a and b", "a".attr && "b".attr) // Or - assertEqual("a or b", 'a || 'b) + assertEqual("a or b", "a".attr || "b".attr) // Combination And/Or check precedence - assertEqual("a and b or c and d", ('a && 'b) || ('c && 'd)) - assertEqual("a or b or c and d", 'a || 'b || ('c && 'd)) + assertEqual("a and b or c and d", ("a".attr && "b".attr) || ("c".attr && "d".attr)) + assertEqual("a or b or c and d", "a".attr || "b".attr || ("c".attr && "d".attr)) // Multiple AND/OR get converted into a balanced tree - assertEqual("a or b or c or d or e or f", (('a || 'b) || 'c) || (('d || 'e) || 'f)) - assertEqual("a and b and c and d and e and f", (('a && 'b) && 'c) && (('d && 'e) && 'f)) + assertEqual("a or b or c or d or e or f", + (("a".attr || "b".attr) || "c".attr) || (("d".attr || "e".attr) || "f".attr)) + assertEqual("a and b and c and d and e and f", + (("a".attr && "b".attr) && "c".attr) && (("d".attr && "e".attr) && "f".attr)) } test("long binary logical expressions") { @@ -125,76 +127,77 @@ class ExpressionParserSuite extends AnalysisTest { } test("not expressions") { - assertEqual("not a", !'a) - assertEqual("!a", !'a) + assertEqual("not a", !"a".attr) + assertEqual("!a", !"a".attr) assertEqual("not true > true", Not(GreaterThan(true, true))) } test("exists expression") { assertEqual( "exists (select 1 from b where b.x = a.x)", - Exists(table("b").where(Symbol("b.x") === Symbol("a.x")).select(1))) + Exists(table("b").where("b.x".attr === "a.x".attr).select(1))) } test("comparison expressions") { - assertEqual("a = b", 'a === 'b) - assertEqual("a == b", 'a === 'b) - assertEqual("a <=> b", 'a <=> 'b) - assertEqual("a <> b", 'a =!= 'b) - assertEqual("a != b", 'a =!= 'b) - assertEqual("a < b", 'a < 'b) - assertEqual("a <= b", 'a <= 'b) - assertEqual("a !> b", 'a <= 'b) - assertEqual("a > b", 'a > 'b) - assertEqual("a >= b", 'a >= 'b) - assertEqual("a !< b", 'a >= 'b) + assertEqual("a = b", "a".attr === "b".attr) + assertEqual("a == b", "a".attr === "b".attr) + assertEqual("a <=> b", "a".attr <=> "b".attr) + assertEqual("a <> b", "a".attr =!= "b".attr) + assertEqual("a != b", "a".attr =!= "b".attr) + assertEqual("a < b", "a".attr < "b".attr) + assertEqual("a <= b", "a".attr <= "b".attr) + assertEqual("a !> b", "a".attr <= "b".attr) + assertEqual("a > b", "a".attr > "b".attr) + assertEqual("a >= b", "a".attr >= "b".attr) + assertEqual("a !< b", "a".attr >= "b".attr) } test("between expressions") { - assertEqual("a between b and c", 'a >= 'b && 'a <= 'c) - assertEqual("a not between b and c", !('a >= 'b && 'a <= 'c)) + assertEqual("a between b and c", "a".attr >= "b".attr && "a".attr <= "c".attr) + assertEqual("a not between b and c", !("a".attr >= "b".attr && "a".attr <= "c".attr)) } test("in expressions") { - assertEqual("a in (b, c, d)", 'a in ('b, 'c, 'd)) - assertEqual("a not in (b, c, d)", !('a in ('b, 'c, 'd))) + assertEqual("a in (b, c, d)", "a".attr in ("b".attr, "c".attr, "d".attr)) + assertEqual("a not in (b, c, d)", !("a".attr in ("b".attr, "c".attr, "d".attr))) } test("in sub-query") { assertEqual( "a in (select b from c)", - InSubquery(Seq('a), ListQuery(table("c").select('b)))) + InSubquery(Seq("a".attr), ListQuery(table("c").select("b".attr)))) assertEqual( "(a, b, c) in (select d, e, f from g)", - InSubquery(Seq('a, 'b, 'c), ListQuery(table("g").select('d, 'e, 'f)))) + InSubquery(Seq("a".attr, "b".attr, "c".attr), + ListQuery(table("g").select("d".attr, "e".attr, "f".attr)))) assertEqual( "(a, b) in (select c from d)", - InSubquery(Seq('a, 'b), ListQuery(table("d").select('c)))) + InSubquery(Seq("a".attr, "b".attr), ListQuery(table("d").select("c".attr)))) assertEqual( "(a) in (select b from c)", - InSubquery(Seq('a), ListQuery(table("c").select('b)))) + InSubquery(Seq("a".attr), ListQuery(table("c").select("b".attr)))) } test("like expressions") { - assertEqual("a like 'pattern%'", 'a like "pattern%") - assertEqual("a not like 'pattern%'", !('a like "pattern%")) - assertEqual("a rlike 'pattern%'", 'a rlike "pattern%") - assertEqual("a not rlike 'pattern%'", !('a rlike "pattern%")) - assertEqual("a regexp 'pattern%'", 'a rlike "pattern%") - assertEqual("a not regexp 'pattern%'", !('a rlike "pattern%")) + assertEqual("a like 'pattern%'", "a".attr like "pattern%") + assertEqual("a not like 'pattern%'", !("a".attr like "pattern%")) + assertEqual("a rlike 'pattern%'", "a".attr rlike "pattern%") + assertEqual("a not rlike 'pattern%'", !("a".attr rlike "pattern%")) + assertEqual("a regexp 'pattern%'", "a".attr rlike "pattern%") + assertEqual("a not regexp 'pattern%'", !("a".attr rlike "pattern%")) } test("like escape expressions") { val message = "Escape string must contain only one character." - assertEqual("a like 'pattern%' escape '#'", 'a.like("pattern%", '#')) - assertEqual("a like 'pattern%' escape '\"'", 'a.like("pattern%", '\"')) + assertEqual("a like 'pattern%' escape '#'", "a".attr.like("pattern%", '#')) + assertEqual("a like 'pattern%' escape '\"'", "a".attr.like("pattern%", '\"')) intercept("a like 'pattern%' escape '##'", message) intercept("a like 'pattern%' escape ''", message) - assertEqual("a not like 'pattern%' escape '#'", !('a.like("pattern%", '#'))) - assertEqual("a not like 'pattern%' escape '\"'", !('a.like("pattern%", '\"'))) + assertEqual("a not like 'pattern%' escape '#'", !("a".attr.like("pattern%", '#'))) + assertEqual("a not like 'pattern%' escape '\"'", !("a".attr.like("pattern%", '\"'))) intercept("a not like 'pattern%' escape '\"/'", message) intercept("a not like 'pattern%' escape ''", message) } @@ -202,21 +205,21 @@ class ExpressionParserSuite extends AnalysisTest { test("like expressions with ESCAPED_STRING_LITERALS = true") { withSQLConf(SQLConf.ESCAPED_STRING_LITERALS.key -> "true") { val parser = new CatalystSqlParser() - assertEqual("a rlike '^\\x20[\\x20-\\x23]+$'", 'a rlike "^\\x20[\\x20-\\x23]+$", parser) - assertEqual("a rlike 'pattern\\\\'", 'a rlike "pattern\\\\", parser) - assertEqual("a rlike 'pattern\\t\\n'", 'a rlike "pattern\\t\\n", parser) + assertEqual("a rlike '^\\x20[\\x20-\\x23]+$'", "a".attr rlike "^\\x20[\\x20-\\x23]+$", parser) + assertEqual("a rlike 'pattern\\\\'", "a".attr rlike "pattern\\\\", parser) + assertEqual("a rlike 'pattern\\t\\n'", "a".attr rlike "pattern\\t\\n", parser) } } test("(NOT) LIKE (ANY | SOME | ALL) expressions") { Seq("any", "some").foreach { quantifier => - assertEqual(s"a like $quantifier ('foo%', 'b%')", 'a likeAny("foo%", "b%")) - assertEqual(s"a not like $quantifier ('foo%', 'b%')", 'a notLikeAny("foo%", "b%")) - assertEqual(s"not (a like $quantifier ('foo%', 'b%'))", !('a likeAny("foo%", "b%"))) + assertEqual(s"a like $quantifier ('foo%', 'b%')", "a".attr likeAny("foo%", "b%")) + assertEqual(s"a not like $quantifier ('foo%', 'b%')", "a".attr notLikeAny("foo%", "b%")) + assertEqual(s"not (a like $quantifier ('foo%', 'b%'))", !("a".attr likeAny("foo%", "b%"))) } - assertEqual("a like all ('foo%', 'b%')", 'a likeAll("foo%", "b%")) - assertEqual("a not like all ('foo%', 'b%')", 'a notLikeAll("foo%", "b%")) - assertEqual("not (a like all ('foo%', 'b%'))", !('a likeAll("foo%", "b%"))) + assertEqual("a like all ('foo%', 'b%')", "a".attr likeAll("foo%", "b%")) + assertEqual("a not like all ('foo%', 'b%')", "a".attr notLikeAll("foo%", "b%")) + assertEqual("not (a like all ('foo%', 'b%'))", !("a".attr likeAll("foo%", "b%"))) Seq("any", "some", "all").foreach { quantifier => intercept(s"a like $quantifier()", "Expected something between '(' and ')'") @@ -224,73 +227,74 @@ class ExpressionParserSuite extends AnalysisTest { } test("is null expressions") { - assertEqual("a is null", 'a.isNull) - assertEqual("a is not null", 'a.isNotNull) - assertEqual("a = b is null", ('a === 'b).isNull) - assertEqual("a = b is not null", ('a === 'b).isNotNull) + assertEqual("a is null", "a".attr.isNull) + assertEqual("a is not null", "a".attr.isNotNull) + assertEqual("a = b is null", ("a".attr === "b".attr).isNull) + assertEqual("a = b is not null", ("a".attr === "b".attr).isNotNull) } test("is distinct expressions") { - assertEqual("a is distinct from b", !('a <=> 'b)) - assertEqual("a is not distinct from b", 'a <=> 'b) + assertEqual("a is distinct from b", !("a".attr <=> "b".attr)) + assertEqual("a is not distinct from b", "a".attr <=> "b".attr) } test("binary arithmetic expressions") { // Simple operations - assertEqual("a * b", 'a * 'b) - assertEqual("a / b", 'a / 'b) - assertEqual("a DIV b", 'a div 'b) - assertEqual("a % b", 'a % 'b) - assertEqual("a + b", 'a + 'b) - assertEqual("a - b", 'a - 'b) - assertEqual("a & b", 'a & 'b) - assertEqual("a ^ b", 'a ^ 'b) - assertEqual("a | b", 'a | 'b) + assertEqual("a * b", "a".attr * "b".attr) + assertEqual("a / b", "a".attr / "b".attr) + assertEqual("a DIV b", "a".attr div "b".attr) + assertEqual("a % b", "a".attr % "b".attr) + assertEqual("a + b", "a".attr + "b".attr) + assertEqual("a - b", "a".attr - "b".attr) + assertEqual("a & b", "a".attr & "b".attr) + assertEqual("a ^ b", "a".attr ^ "b".attr) + assertEqual("a | b", "a".attr | "b".attr) // Check precedences assertEqual( "a * t | b ^ c & d - e + f % g DIV h / i * k", - 'a * 't | ('b ^ ('c & ('d - 'e + (('f % 'g div 'h) / 'i * 'k))))) + "a".attr * "t".attr | ("b".attr ^ ("c".attr & ("d".attr - "e".attr + + (("f".attr % "g".attr div "h".attr) / "i".attr * "k".attr))))) } test("unary arithmetic expressions") { - assertEqual("+a", +'a) - assertEqual("-a", -'a) - assertEqual("~a", ~'a) - assertEqual("-+~~a", -( +(~(~'a)))) + assertEqual("+a", +"a".attr) + assertEqual("-a", -"a".attr) + assertEqual("~a", ~"a".attr) + assertEqual("-+~~a", -( +(~(~"a".attr)))) } test("cast expressions") { // Note that DataType parsing is tested elsewhere. - assertEqual("cast(a as int)", 'a.cast(IntegerType)) - assertEqual("cast(a as timestamp)", 'a.cast(TimestampType)) - assertEqual("cast(a as array)", 'a.cast(ArrayType(IntegerType))) - assertEqual("cast(cast(a as int) as long)", 'a.cast(IntegerType).cast(LongType)) + assertEqual("cast(a as int)", "a".attr.cast(IntegerType)) + assertEqual("cast(a as timestamp)", "a".attr.cast(TimestampType)) + assertEqual("cast(a as array)", "a".attr.cast(ArrayType(IntegerType))) + assertEqual("cast(cast(a as int) as long)", "a".attr.cast(IntegerType).cast(LongType)) } test("function expressions") { - assertEqual("foo()", 'foo.function()) + assertEqual("foo()", "foo".attr.function()) assertEqual("foo.bar()", UnresolvedFunction(FunctionIdentifier("bar", Some("foo")), Seq.empty, isDistinct = false)) - assertEqual("foo(*)", 'foo.function(star())) - assertEqual("count(*)", 'count.function(1)) - assertEqual("foo(a, b)", 'foo.function('a, 'b)) - assertEqual("foo(all a, b)", 'foo.function('a, 'b)) - assertEqual("foo(distinct a, b)", 'foo.distinctFunction('a, 'b)) - assertEqual("grouping(distinct a, b)", 'grouping.distinctFunction('a, 'b)) - assertEqual("`select`(all a, b)", 'select.function('a, 'b)) + assertEqual("foo(*)", "foo".attr.function(star())) + assertEqual("count(*)", "count".attr.function(1)) + assertEqual("foo(a, b)", "foo".attr.function("a".attr, "b".attr)) + assertEqual("foo(all a, b)", "foo".attr.function("a".attr, "b".attr)) + assertEqual("foo(distinct a, b)", "foo".attr.distinctFunction("a".attr, "b".attr)) + assertEqual("grouping(distinct a, b)", "grouping".attr.distinctFunction("a".attr, "b".attr)) + assertEqual("`select`(all a, b)", "select".attr.function("a".attr, "b".attr)) intercept("foo(a x)", "extraneous input 'x'") } - private def lv(s: Symbol) = UnresolvedNamedLambdaVariable(Seq(s.name)) + private def lv(s: String) = UnresolvedNamedLambdaVariable(Seq(s)) test("lambda functions") { - assertEqual("x -> x + 1", LambdaFunction(lv('x) + 1, Seq(lv('x)))) - assertEqual("(x, y) -> x + y", LambdaFunction(lv('x) + lv('y), Seq(lv('x), lv('y)))) + assertEqual("x -> x + 1", LambdaFunction(lv("x") + 1, Seq(lv("x")))) + assertEqual("(x, y) -> x + y", LambdaFunction(lv("x") + lv("y"), Seq(lv("x"), lv("y")))) } test("window function expressions") { - val func = 'foo.function(star()) + val func = "foo".attr.function(star()) def windowed( partitioning: Seq[Expression] = Seq.empty, ordering: Seq[SortOrder] = Seq.empty, @@ -301,27 +305,31 @@ class ExpressionParserSuite extends AnalysisTest { // Basic window testing. assertEqual("foo(*) over w1", UnresolvedWindowExpression(func, WindowSpecReference("w1"))) assertEqual("foo(*) over ()", windowed()) - assertEqual("foo(*) over (partition by a, b)", windowed(Seq('a, 'b))) - assertEqual("foo(*) over (distribute by a, b)", windowed(Seq('a, 'b))) - assertEqual("foo(*) over (cluster by a, b)", windowed(Seq('a, 'b))) - assertEqual("foo(*) over (order by a desc, b asc)", windowed(Seq.empty, Seq('a.desc, 'b.asc))) - assertEqual("foo(*) over (sort by a desc, b asc)", windowed(Seq.empty, Seq('a.desc, 'b.asc))) - assertEqual("foo(*) over (partition by a, b order by c)", windowed(Seq('a, 'b), Seq('c.asc))) - assertEqual("foo(*) over (distribute by a, b sort by c)", windowed(Seq('a, 'b), Seq('c.asc))) + assertEqual("foo(*) over (partition by a, b)", windowed(Seq("a".attr, "b".attr))) + assertEqual("foo(*) over (distribute by a, b)", windowed(Seq("a".attr, "b".attr))) + assertEqual("foo(*) over (cluster by a, b)", windowed(Seq("a".attr, "b".attr))) + assertEqual("foo(*) over (order by a desc, b asc)", + windowed(Seq.empty, Seq("a".attr.desc, "b".attr.asc))) + assertEqual("foo(*) over (sort by a desc, b asc)", + windowed(Seq.empty, Seq("a".attr.desc, "b".attr.asc))) + assertEqual("foo(*) over (partition by a, b order by c)", + windowed(Seq("a".attr, "b".attr), Seq("c".attr.asc))) + assertEqual("foo(*) over (distribute by a, b sort by c)", + windowed(Seq("a".attr, "b".attr), Seq("c".attr.asc))) // Test use of expressions in window functions. assertEqual( "sum(product + 1) over (partition by ((product) + (1)) order by 2)", - WindowExpression('sum.function('product + 1), - WindowSpecDefinition(Seq('product + 1), Seq(Literal(2).asc), UnspecifiedFrame))) + WindowExpression("sum".attr.function("product".attr + 1), + WindowSpecDefinition(Seq("product".attr + 1), Seq(Literal(2).asc), UnspecifiedFrame))) assertEqual( "sum(product + 1) over (partition by ((product / 2) + 1) order by 2)", - WindowExpression('sum.function('product + 1), - WindowSpecDefinition(Seq('product / 2 + 1), Seq(Literal(2).asc), UnspecifiedFrame))) + WindowExpression("sum".attr.function("product".attr + 1), + WindowSpecDefinition(Seq("product".attr / 2 + 1), Seq(Literal(2).asc), UnspecifiedFrame))) } test("range/rows window function expressions") { - val func = 'foo.function(star()) + val func = "foo".attr.function(star()) def windowed( partitioning: Seq[Expression] = Seq.empty, ordering: Seq[SortOrder] = Seq.empty, @@ -380,7 +388,8 @@ class ExpressionParserSuite extends AnalysisTest { boundaries.foreach { case (boundarySql, begin, end) => val query = s"foo(*) over (partition by a order by b $frameTypeSql $boundarySql)" - val expr = windowed(Seq('a), Seq('b.asc), SpecifiedWindowFrame(frameType, begin, end)) + val expr = windowed( + Seq("a".attr), Seq("b".attr.asc), SpecifiedWindowFrame(frameType, begin, end)) assertEqual(query, expr) } } @@ -392,65 +401,66 @@ class ExpressionParserSuite extends AnalysisTest { test("row constructor") { // Note that '(a)' will be interpreted as a nested expression. - assertEqual("(a, b)", CreateStruct(Seq('a, 'b))) - assertEqual("(a, b, c)", CreateStruct(Seq('a, 'b, 'c))) - assertEqual("(a as b, b as c)", CreateStruct(Seq('a as 'b, 'b as 'c))) + assertEqual("(a, b)", CreateStruct(Seq("a".attr, "b".attr))) + assertEqual("(a, b, c)", CreateStruct(Seq("a".attr, "b".attr, "c".attr))) + assertEqual("(a as b, b as c)", CreateStruct(Seq("a".attr as "b", "b".attr as "c"))) } test("scalar sub-query") { assertEqual( "(select max(val) from tbl) > current", - ScalarSubquery(table("tbl").select('max.function('val))) > 'current) + ScalarSubquery(table("tbl").select("max".attr.function("val".attr))) > "current".attr) assertEqual( "a = (select b from s)", - 'a === ScalarSubquery(table("s").select('b))) + "a".attr === ScalarSubquery(table("s").select("b".attr))) } test("case when") { assertEqual("case a when 1 then b when 2 then c else d end", - CaseKeyWhen('a, Seq(1, 'b, 2, 'c, 'd))) + CaseKeyWhen("a".attr, Seq(1, "b".attr, 2, "c".attr, "d".attr))) assertEqual("case (a or b) when true then c when false then d else e end", - CaseKeyWhen('a || 'b, Seq(true, 'c, false, 'd, 'e))) + CaseKeyWhen("a".attr || "b".attr, Seq(true, "c".attr, false, "d".attr, "e".attr))) assertEqual("case 'a'='a' when true then 1 end", CaseKeyWhen("a" === "a", Seq(true, 1))) assertEqual("case when a = 1 then b when a = 2 then c else d end", - CaseWhen(Seq(('a === 1, 'b.expr), ('a === 2, 'c.expr)), 'd)) + CaseWhen(Seq(("a".attr === 1, "b".attr.expr), ("a".attr === 2, "c".attr.expr)), "d".attr)) assertEqual("case when (1) + case when a > b then c else d end then f else g end", - CaseWhen(Seq((Literal(1) + CaseWhen(Seq(('a > 'b, 'c.expr)), 'd.expr), 'f.expr)), 'g)) + CaseWhen(Seq((Literal(1) + CaseWhen(Seq(("a".attr > "b".attr, "c".attr.expr)), + "d".attr.expr), "f".attr.expr)), "g".attr)) } test("dereference") { assertEqual("a.b", UnresolvedAttribute("a.b")) assertEqual("`select`.b", UnresolvedAttribute("select.b")) - assertEqual("(a + b).b", ('a + 'b).getField("b")) // This will fail analysis. + assertEqual("(a + b).b", ("a".attr + "b".attr).getField("b")) // This will fail analysis. assertEqual( "struct(a, b).b", - namedStruct(NamePlaceholder, 'a, NamePlaceholder, 'b).getField("b")) + namedStruct(NamePlaceholder, "a".attr, NamePlaceholder, "b".attr).getField("b")) } test("reference") { // Regular - assertEqual("a", 'a) + assertEqual("a", "a".attr) // Starting with a digit. - assertEqual("1a", Symbol("1a")) + assertEqual("1a", "1a".attr) // Quoted using a keyword. - assertEqual("`select`", 'select) + assertEqual("`select`", "select".attr) // Unquoted using an unreserved keyword. - assertEqual("columns", 'columns) + assertEqual("columns", "columns".attr) } test("subscript") { - assertEqual("a[b]", 'a.getItem('b)) - assertEqual("a[1 + 1]", 'a.getItem(Literal(1) + 1)) - assertEqual("`c`.a[b]", UnresolvedAttribute("c.a").getItem('b)) + assertEqual("a[b]", "a".attr.getItem("b".attr)) + assertEqual("a[1 + 1]", "a".attr.getItem(Literal(1) + 1)) + assertEqual("`c`.a[b]", UnresolvedAttribute("c.a").getItem("b".attr)) } test("parenthesis") { - assertEqual("(a)", 'a) - assertEqual("r * (a + b)", 'r * ('a + 'b)) + assertEqual("(a)", "a".attr) + assertEqual("r * (a + b)", "r".attr * ("a".attr + "b".attr)) } test("type constructors") { @@ -759,7 +769,8 @@ class ExpressionParserSuite extends AnalysisTest { test("composed expressions") { assertEqual("1 + r.r As q", (Literal(1) + UnresolvedAttribute("r.r")).as("q")) - assertEqual("1 - f('o', o(bar))", Literal(1) - 'f.function("o", 'o.function('bar))) + assertEqual("1 - f('o', o(bar))", + Literal(1) - "f".attr.function("o", "o".attr.function("bar".attr))) intercept("1 - f('o', o(bar)) hello * world", "mismatched input '*'") } @@ -786,10 +797,10 @@ class ExpressionParserSuite extends AnalysisTest { } test("SPARK-19526 Support ignore nulls keywords for first and last") { - assertEqual("first(a ignore nulls)", First('a, true).toAggregateExpression()) - assertEqual("first(a)", First('a, false).toAggregateExpression()) - assertEqual("last(a ignore nulls)", Last('a, true).toAggregateExpression()) - assertEqual("last(a)", Last('a, false).toAggregateExpression()) + assertEqual("first(a ignore nulls)", First("a".attr, true).toAggregateExpression()) + assertEqual("first(a)", First("a".attr, false).toAggregateExpression()) + assertEqual("last(a ignore nulls)", Last("a".attr, true).toAggregateExpression()) + assertEqual("last(a)", Last("a".attr, false).toAggregateExpression()) } test("timestamp literals") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 1ca2f5226903..ff41474e5c97 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -208,23 +208,25 @@ class PlanParserSuite extends AnalysisTest { test("simple select query") { assertEqual("select 1", OneRowRelation().select(1)) - assertEqual("select a, b", OneRowRelation().select('a, 'b)) - assertEqual("select a, b from db.c", table("db", "c").select('a, 'b)) - assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b)) + assertEqual("select a, b", OneRowRelation().select("a".attr, "b".attr)) + assertEqual("select a, b from db.c", table("db", "c").select("a".attr, "b".attr)) + assertEqual("select a, b from db.c where x < 1", + table("db", "c").where("x".attr < 1).select("a".attr, "b".attr)) assertEqual( "select a, b from db.c having x < 1", - table("db", "c").having()('a, 'b)('x < 1)) - assertEqual("select distinct a, b from db.c", Distinct(table("db", "c").select('a, 'b))) - assertEqual("select all a, b from db.c", table("db", "c").select('a, 'b)) - assertEqual("select from tbl", OneRowRelation().select('from.as("tbl"))) - assertEqual("select a from 1k.2m", table("1k", "2m").select('a)) + table("db", "c").having()("a".attr, "b".attr)("x".attr < 1)) + assertEqual("select distinct a, b from db.c", + Distinct(table("db", "c").select("a".attr, "b".attr))) + assertEqual("select all a, b from db.c", table("db", "c").select("a".attr, "b".attr)) + assertEqual("select from tbl", OneRowRelation().select("from".attr.as("tbl"))) + assertEqual("select a from 1k.2m", table("1k", "2m").select("a".attr)) } test("hive-style single-FROM statement") { - assertEqual("from a select b, c", table("a").select('b, 'c)) - assertEqual( - "from db.a select b, c where d < 1", table("db", "a").where('d < 1).select('b, 'c)) - assertEqual("from a select distinct b, c", Distinct(table("a").select('b, 'c))) + assertEqual("from a select b, c", table("a").select("b".attr, "c".attr)) + assertEqual("from db.a select b, c where d < 1", + table("db", "a").where("d".attr < 1).select("b".attr, "c".attr)) + assertEqual("from a select distinct b, c", Distinct(table("a").select("b".attr, "c".attr))) // Weird "FROM table" queries, should be invalid anyway intercept("from a", "no viable alternative at input 'from a'") @@ -234,7 +236,7 @@ class PlanParserSuite extends AnalysisTest { test("multi select query") { assertEqual( "from a select * select * where s < 10", - table("a").select(star()).union(table("a").where('s < 10).select(star()))) + table("a").select(star()).union(table("a").where("s".attr < 10).select(star()))) intercept( "from a select * select * from x where a.s < 10", "mismatched input 'from' expecting") @@ -244,7 +246,7 @@ class PlanParserSuite extends AnalysisTest { assertEqual( "from a insert into tbl1 select * insert into tbl2 select * where s < 10", table("a").select(star()).insertInto("tbl1").union( - table("a").where('s < 10).select(star()).insertInto("tbl2"))) + table("a").where("s".attr < 10).select(star()).insertInto("tbl2"))) assertEqual( "select * from (from a select * select *)", table("a").select(star()) @@ -267,8 +269,8 @@ class PlanParserSuite extends AnalysisTest { val orderSortDistrClusterClauses = Seq( ("", basePlan), - (" order by a, b desc", basePlan.orderBy('a.asc, 'b.desc)), - (" sort by a, b desc", basePlan.sortBy('a.asc, 'b.desc)) + (" order by a, b desc", basePlan.orderBy("a".attr.asc, "b".attr.desc)), + (" sort by a, b desc", basePlan.sortBy("a".attr.asc, "b".attr.desc)) ) orderSortDistrClusterClauses.foreach { @@ -308,7 +310,7 @@ class PlanParserSuite extends AnalysisTest { insert(Map("c" -> Option("d"), "e" -> Option("1")))) // Multi insert - val plan2 = table("t").where('x > 5).select(star()) + val plan2 = table("t").where("x".attr > 5).select(star()) assertEqual("from t insert into s select * limit 1 insert into u select * where x > 5", plan.limit(1).insertInto("s").union(plan2.insertInto("u"))) } @@ -317,20 +319,26 @@ class PlanParserSuite extends AnalysisTest { val sql = "select a, b, sum(c) as c from d group by a, b" // Normal - assertEqual(sql, table("d").groupBy('a, 'b)('a, 'b, 'sum.function('c).as("c"))) + assertEqual(sql, table("d").groupBy( + "a".attr, "b".attr)("a".attr, "b".attr, "sum".attr.function("c".attr).as("c"))) // Cube assertEqual(s"$sql with cube", - table("d").groupBy(Cube(Seq('a, 'b)))('a, 'b, 'sum.function('c).as("c"))) + table("d").groupBy( + Cube(Seq("a".attr, "b".attr)))("a".attr, "b".attr, "sum".attr.function("c".attr).as("c"))) // Rollup assertEqual(s"$sql with rollup", - table("d").groupBy(Rollup(Seq('a, 'b)))('a, 'b, 'sum.function('c).as("c"))) + table("d").groupBy(Rollup( + Seq("a".attr, "b".attr)))("a".attr, "b".attr, "sum".attr.function("c".attr).as("c"))) // Grouping Sets assertEqual(s"$sql grouping sets((a, b), (a), ())", - GroupingSets(Seq(Seq('a, 'b), Seq('a), Seq()), Seq('a, 'b), table("d"), - Seq('a, 'b, 'sum.function('c).as("c")))) + GroupingSets(Seq( + Seq("a".attr, "b".attr), + Seq("a".attr), Seq()), + Seq("a".attr, "b".attr), table("d"), + Seq("a".attr, "b".attr, "sum".attr.function("c".attr).as("c")))) val m = intercept[ParseException] { parsePlan("SELECT a, b, count(distinct a, distinct b) as c FROM d GROUP BY a, b") @@ -350,7 +358,7 @@ class PlanParserSuite extends AnalysisTest { // Note that WindowSpecs are testing in the ExpressionParserSuite val sql = "select * from t" val plan = table("t").select(star()) - val spec = WindowSpecDefinition(Seq('a, 'b), Seq('c.asc), + val spec = WindowSpecDefinition(Seq("a".attr, "b".attr), Seq("c".attr.asc), SpecifiedWindowFrame(RowFrame, -Literal(1), Literal(1))) // Test window resolution. @@ -376,8 +384,8 @@ class PlanParserSuite extends AnalysisTest { } test("lateral view") { - val explode = UnresolvedGenerator(FunctionIdentifier("explode"), Seq('x)) - val jsonTuple = UnresolvedGenerator(FunctionIdentifier("json_tuple"), Seq('x, 'y)) + val explode = UnresolvedGenerator(FunctionIdentifier("explode"), Seq("x".attr)) + val jsonTuple = UnresolvedGenerator(FunctionIdentifier("json_tuple"), Seq("x".attr, "y".attr)) // Single lateral view assertEqual( @@ -413,12 +421,12 @@ class PlanParserSuite extends AnalysisTest { .generate(jsonTuple, alias = Some("jtup"), outputNames = Seq("q", "z")) .select(star()) .insertInto("t2"), - from.where('s < 10).select(star()).insertInto("t3"))) + from.where("s".attr < 10).select(star()).insertInto("t3"))) // Unresolved generator. val expected = table("t") .generate( - UnresolvedGenerator(FunctionIdentifier("posexplode"), Seq('x)), + UnresolvedGenerator(FunctionIdentifier("posexplode"), Seq("x".attr)), alias = Some("posexpl"), outputNames = Seq("x", "y")) .select(star()) @@ -447,7 +455,7 @@ class PlanParserSuite extends AnalysisTest { val testConditionalJoin = (sql: String, jt: JoinType) => { assertEqual( s"select * from t $sql u as uu on a = b", - table("t").join(table("u").as("uu"), jt, Option('a === 'b)).select(star())) + table("t").join(table("u").as("uu"), jt, Option("a".attr === "b".attr)).select(star())) } val testNaturalJoin = (sql: String, jt: JoinType) => { assertEqual( @@ -495,8 +503,8 @@ class PlanParserSuite extends AnalysisTest { "select * from t1 cross join t2 join t3 on t3.id = t1.id join t4 on t4.id = t1.id", table("t1") .join(table("t2"), Cross) - .join(table("t3"), Inner, Option(Symbol("t3.id") === Symbol("t1.id"))) - .join(table("t4"), Inner, Option(Symbol("t4.id") === Symbol("t1.id"))) + .join(table("t3"), Inner, Option("t3.id".attr === "t1.id".attr)) + .join(table("t4"), Inner, Option("t4.id".attr === "t1.id".attr)) .select(star())) // Test multiple on clauses. @@ -507,17 +515,20 @@ class PlanParserSuite extends AnalysisTest { "select * from t1 inner join (t2 inner join t3 on col3 = col2) on col3 = col1", table("t1") .join(table("t2") - .join(table("t3"), Inner, Option('col3 === 'col2)), Inner, Option('col3 === 'col1)) + .join(table("t3"), Inner, + Option("col3".attr === "col2".attr)), Inner, Option("col3".attr === "col1".attr)) .select(star())) assertEqual( "select * from t1 inner join (t2 inner join t3) on col3 = col2", table("t1") - .join(table("t2").join(table("t3"), Inner, None), Inner, Option('col3 === 'col2)) + .join(table("t2").join(table("t3"), Inner, None), Inner, + Option("col3".attr === "col2".attr)) .select(star())) assertEqual( "select * from t1 inner join (t2 inner join t3 on col3 = col2)", table("t1") - .join(table("t2").join(table("t3"), Inner, Option('col3 === 'col2)), Inner, None) + .join(table("t2").join(table("t3"), Inner, + Option("col3".attr === "col2".attr)), Inner, None) .select(star())) // Implicit joins. @@ -525,7 +536,7 @@ class PlanParserSuite extends AnalysisTest { "select * from t1, t3 join t2 on t1.col1 = t2.col2", table("t1") .join(table("t3")) - .join(table("t2"), Inner, Option(Symbol("t1.col1") === Symbol("t2.col2"))) + .join(table("t2"), Inner, Option("t1.col1".attr === "t2.col2".attr)) .select(star())) } @@ -548,7 +559,7 @@ class PlanParserSuite extends AnalysisTest { } test("sub-query") { - val plan = table("t0").select('id) + val plan = table("t0").select("id".attr) assertEqual("select id from (t0)", plan) assertEqual("select id from ((((((t0))))))", plan) assertEqual( @@ -566,20 +577,20 @@ class PlanParserSuite extends AnalysisTest { | union all | (select id from t0)) as u_1 """.stripMargin, - plan.union(plan).union(plan).as("u_1").select('id)) + plan.union(plan).union(plan).as("u_1").select("id".attr)) } test("scalar sub-query") { assertEqual( "select (select max(b) from s) ss from t", - table("t").select(ScalarSubquery(table("s").select('max.function('b))).as("ss"))) + table("t").select(ScalarSubquery(table("s").select("max".attr.function("b".attr))).as("ss"))) assertEqual( "select * from t where a = (select b from s)", - table("t").where('a === ScalarSubquery(table("s").select('b))).select(star())) + table("t").where("a".attr === ScalarSubquery(table("s").select("b".attr))).select(star())) assertEqual( "select g from t group by g having a > (select b from s)", table("t") - .having('g)('g)('a > ScalarSubquery(table("s").select('b)))) + .having("g".attr)("g".attr)("a".attr > ScalarSubquery(table("s").select("b".attr)))) } test("table reference") { @@ -623,7 +634,7 @@ class PlanParserSuite extends AnalysisTest { "t", UnresolvedSubqueryColumnAliases( Seq("col1", "col2"), - UnresolvedRelation(TableIdentifier("t")).select('a.as("x"), 'b.as("y")) + UnresolvedRelation(TableIdentifier("t")).select("a".attr.as("x"), "b".attr.as("y")) ) ).select(star())) } @@ -637,7 +648,7 @@ class PlanParserSuite extends AnalysisTest { "dst", UnresolvedSubqueryColumnAliases( Seq("a", "b", "c", "d"), - src1.join(src2, Inner, Option(Symbol("s1.id") === Symbol("s2.id"))) + src1.join(src2, Inner, Option("s1.id".attr === "s2.id".attr)) ) ).select(star())) } @@ -649,7 +660,7 @@ class PlanParserSuite extends AnalysisTest { "t", UnresolvedSubqueryColumnAliases( Seq("col1", "col2"), - UnresolvedRelation(TableIdentifier("t")).select('a.as("x"), 'b.as("y"))) + UnresolvedRelation(TableIdentifier("t")).select("a".attr.as("x"), "b".attr.as("y"))) ).select($"t.col1", $"t.col2") ) } @@ -668,10 +679,10 @@ class PlanParserSuite extends AnalysisTest { test("simple select query with !> and !<") { // !< is equivalent to >= assertEqual("select a, b from db.c where x !< 1", - table("db", "c").where('x >= 1).select('a, 'b)) + table("db", "c").where("x".attr >= 1).select("a".attr, "b".attr)) // !> is equivalent to <= assertEqual("select a, b from db.c where x !> 1", - table("db", "c").where('x <= 1).select('a, 'b)) + table("db", "c").where("x".attr <= 1).select("a".attr, "b".attr)) } test("select hint syntax") { @@ -715,7 +726,7 @@ class PlanParserSuite extends AnalysisTest { comparePlans( parsePlan("SELECT /*+ MAPJOIN(t) */ a from t where true group by a order by a"), UnresolvedHint("MAPJOIN", Seq($"t"), - table("t").where(Literal(true)).groupBy('a)('a)).orderBy('a.asc)) + table("t").where(Literal(true)).groupBy("a".attr)("a".attr)).orderBy("a".attr.asc)) comparePlans( parsePlan("SELECT /*+ COALESCE(10) */ * FROM t"), @@ -1039,14 +1050,14 @@ class PlanParserSuite extends AnalysisTest { test("CTE with column alias") { assertEqual( "WITH t(x) AS (SELECT c FROM a) SELECT * FROM t", - cte(table("t").select(star()), "t" -> ((table("a").select('c), Seq("x"))))) + cte(table("t").select(star()), "t" -> ((table("a").select("c".attr), Seq("x"))))) } test("statement containing terminal semicolons") { assertEqual("select 1;", OneRowRelation().select(1)) - assertEqual("select a, b;", OneRowRelation().select('a, 'b)) - assertEqual("select a, b from db.c;;;", table("db", "c").select('a, 'b)) - assertEqual("select a, b from db.c; ;; ;", table("db", "c").select('a, 'b)) + assertEqual("select a, b;", OneRowRelation().select("a".attr, "b".attr)) + assertEqual("select a, b from db.c;;;", table("db", "c").select("a".attr, "b".attr)) + assertEqual("select a, b from db.c; ;; ;", table("db", "c").select("a".attr, "b".attr)) } test("SPARK-32106: TRANSFORM plan") { @@ -1058,7 +1069,7 @@ class PlanParserSuite extends AnalysisTest { |FROM testData """.stripMargin, ScriptTransformation( - Seq('a, 'b, 'c), + Seq("a".attr, "b".attr, "c".attr), "cat", Seq(AttributeReference("key", StringType)(), AttributeReference("value", StringType)()), @@ -1075,7 +1086,7 @@ class PlanParserSuite extends AnalysisTest { |FROM testData """.stripMargin, ScriptTransformation( - Seq('a, 'b, 'c), + Seq("a".attr, "b".attr, "c".attr), "cat", Seq(AttributeReference("a", StringType)(), AttributeReference("b", StringType)(), @@ -1092,7 +1103,7 @@ class PlanParserSuite extends AnalysisTest { |FROM testData """.stripMargin, ScriptTransformation( - Seq('a, 'b, 'c), + Seq("a".attr, "b".attr, "c".attr), "cat", Seq(AttributeReference("a", IntegerType)(), AttributeReference("b", StringType)(), @@ -1121,7 +1132,7 @@ class PlanParserSuite extends AnalysisTest { |FROM testData """.stripMargin, ScriptTransformation( - Seq('a, 'b, 'c), + Seq("a".attr, "b".attr, "c".attr), "cat", Seq(AttributeReference("a", StringType)(), AttributeReference("b", StringType)(), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala index 5ad748b6113d..5f99c55acede 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala @@ -57,22 +57,22 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { } test("propagating constraints in filters") { - val tr = LocalRelation('a.int, 'b.string, 'c.int) + val tr = LocalRelation("a".attr.int, "b".attr.string, "c".attr.int) assert(tr.analyze.constraints.isEmpty) - assert(tr.where('a.attr > 10).select('c.attr, 'b.attr).analyze.constraints.isEmpty) + assert(tr.where("a".attr > 10).select("c".attr, "b".attr).analyze.constraints.isEmpty) verifyConstraints(tr - .where('a.attr > 10) + .where("a".attr > 10) .analyze.constraints, ExpressionSet(Seq(resolveColumn(tr, "a") > 10, IsNotNull(resolveColumn(tr, "a"))))) verifyConstraints(tr - .where('a.attr > 10) - .select('c.attr, 'a.attr) - .where('c.attr =!= 100) + .where("a".attr > 10) + .select("c".attr, "a".attr) + .where("c".attr =!= 100) .analyze.constraints, ExpressionSet(Seq(resolveColumn(tr, "a") > 10, resolveColumn(tr, "c") =!= 100, @@ -81,12 +81,15 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { } test("propagating constraints in aggregate") { - val tr = LocalRelation('a.int, 'b.string, 'c.int) + val tr = LocalRelation("a".attr.int, "b".attr.string, "c".attr.int) assert(tr.analyze.constraints.isEmpty) - val aliasedRelation = tr.where('c.attr > 10 && 'a.attr < 5) - .groupBy('a, 'c, 'b)('a, 'c.as("c1"), count('a).as("a3")).select('c1, 'a, 'a3).analyze + val aliasedRelation = tr + .where("c".attr > 10 && "a".attr < 5) + .groupBy("a".attr, "c".attr, "b".attr)("a".attr, "c".attr.as("c1"), count("a".attr).as("a3")) + .select("c1".attr, "a".attr, "a3".attr) + .analyze // SPARK-16644: aggregate expression count(a) should not appear in the constraints. verifyConstraints(aliasedRelation.analyze.constraints, @@ -98,13 +101,13 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { } test("propagating constraints in expand") { - val tr = LocalRelation('a.int, 'b.int, 'c.int) + val tr = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) assert(tr.analyze.constraints.isEmpty) - // We add IsNotNull constraints for 'a, 'b and 'c into LocalRelation + // We add IsNotNull constraints for "a".attr, "b".attr and "c".attr into LocalRelation // by creating notNullRelation. - val notNullRelation = tr.where('c.attr > 10 && 'a.attr < 5 && 'b.attr > 2) + val notNullRelation = tr.where("c".attr > 10 && "a".attr < 5 && "b".attr > 2) verifyConstraints(notNullRelation.analyze.constraints, ExpressionSet(Seq(resolveColumn(notNullRelation.analyze, "c") > 10, IsNotNull(resolveColumn(notNullRelation.analyze, "c")), @@ -115,21 +118,24 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { val expand = Expand( Seq( - Seq('c, Literal.create(null, StringType), 1), - Seq('c, 'a, 2)), - Seq('c, 'a, 'gid.int), - Project(Seq('a, 'c), + Seq("c".attr, Literal.create(null, StringType), 1), + Seq("c".attr, "a".attr, 2)), + Seq("c".attr, "a".attr, "gid".attr.int), + Project(Seq("a".attr, "c".attr), notNullRelation)) verifyConstraints(expand.analyze.constraints, ExpressionSet(Seq.empty[Expression])) } test("propagating constraints in aliases") { - val tr = LocalRelation('a.int, 'b.string, 'c.int) + val tr = LocalRelation("a".attr.int, "b".attr.string, "c".attr.int) - assert(tr.where('c.attr > 10).select('a.as('x), 'b.as('y)).analyze.constraints.isEmpty) + assert(tr.where("c".attr > 10).select("a".attr.as("x"), + "b".attr.as("y")).analyze.constraints.isEmpty) - val aliasedRelation = tr.where('a.attr > 10).select('a.as('x), 'b, 'b.as('y), 'a.as('z)) + val aliasedRelation = tr + .where("a".attr > 10) + .select("a".attr.as("x"), "b".attr, "b".attr.as("y"), "a".attr.as("z")) verifyConstraints(aliasedRelation.analyze.constraints, ExpressionSet(Seq(resolveColumn(aliasedRelation.analyze, "x") > 10, @@ -139,7 +145,7 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { resolveColumn(aliasedRelation.analyze, "z") > 10, IsNotNull(resolveColumn(aliasedRelation.analyze, "z"))))) - val multiAlias = tr.where('a === 'c + 10).select('a.as('x), 'c.as('y)) + val multiAlias = tr.where("a".attr === "c".attr + 10).select("a".attr.as("x"), "c".attr.as("y")) verifyConstraints(multiAlias.analyze.constraints, ExpressionSet(Seq(IsNotNull(resolveColumn(multiAlias.analyze, "x")), IsNotNull(resolveColumn(multiAlias.analyze, "y")), @@ -148,46 +154,46 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { } test("propagating constraints in union") { - val tr1 = LocalRelation('a.int, 'b.int, 'c.int) - val tr2 = LocalRelation('d.int, 'e.int, 'f.int) - val tr3 = LocalRelation('g.int, 'h.int, 'i.int) + val tr1 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) + val tr2 = LocalRelation("d".attr.int, "e".attr.int, "f".attr.int) + val tr3 = LocalRelation("g".attr.int, "h".attr.int, "i".attr.int) assert(tr1 - .where('a.attr > 10) - .union(tr2.where('e.attr > 10) - .union(tr3.where('i.attr > 10))) + .where("a".attr > 10) + .union(tr2.where("e".attr > 10) + .union(tr3.where("i".attr > 10))) .analyze.constraints.isEmpty) verifyConstraints(tr1 - .where('a.attr > 10) - .union(tr2.where('d.attr > 10) - .union(tr3.where('g.attr > 10))) + .where("a".attr > 10) + .union(tr2.where("d".attr > 10) + .union(tr3.where("g".attr > 10))) .analyze.constraints, ExpressionSet(Seq(resolveColumn(tr1, "a") > 10, IsNotNull(resolveColumn(tr1, "a"))))) val a = resolveColumn(tr1, "a") verifyConstraints(tr1 - .where('a.attr > 10) - .union(tr2.where('d.attr > 11)) + .where("a".attr > 10) + .union(tr2.where("d".attr > 11)) .analyze.constraints, ExpressionSet(Seq(a > 10 || a > 11, IsNotNull(a)))) val b = resolveColumn(tr1, "b") verifyConstraints(tr1 - .where('a.attr > 10 && 'b.attr < 10) - .union(tr2.where('d.attr > 11 && 'e.attr < 11)) + .where("a".attr > 10 && "b".attr < 10) + .union(tr2.where("d".attr > 11 && "e".attr < 11)) .analyze.constraints, ExpressionSet(Seq(a > 10 || a > 11, b < 10 || b < 11, IsNotNull(a), IsNotNull(b)))) } test("propagating constraints in intersect") { - val tr1 = LocalRelation('a.int, 'b.int, 'c.int) - val tr2 = LocalRelation('a.int, 'b.int, 'c.int) + val tr1 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) + val tr2 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) verifyConstraints(tr1 - .where('a.attr > 10) - .intersect(tr2.where('b.attr < 100), isAll = false) + .where("a".attr > 10) + .intersect(tr2.where("b".attr < 100), isAll = false) .analyze.constraints, ExpressionSet(Seq(resolveColumn(tr1, "a") > 10, resolveColumn(tr1, "b") < 100, @@ -196,22 +202,22 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { } test("propagating constraints in except") { - val tr1 = LocalRelation('a.int, 'b.int, 'c.int) - val tr2 = LocalRelation('a.int, 'b.int, 'c.int) + val tr1 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) + val tr2 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) verifyConstraints(tr1 - .where('a.attr > 10) - .except(tr2.where('b.attr < 100), isAll = false) + .where("a".attr > 10) + .except(tr2.where("b".attr < 100), isAll = false) .analyze.constraints, ExpressionSet(Seq(resolveColumn(tr1, "a") > 10, IsNotNull(resolveColumn(tr1, "a"))))) } test("propagating constraints in inner join") { - val tr1 = LocalRelation('a.int, 'b.int, 'c.int).subquery('tr1) - val tr2 = LocalRelation('a.int, 'd.int, 'e.int).subquery('tr2) + val tr1 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int).subquery("tr1") + val tr2 = LocalRelation("a".attr.int, "d".attr.int, "e".attr.int).subquery("tr2") verifyConstraints(tr1 - .where('a.attr > 10) - .join(tr2.where('d.attr < 100), Inner, Some("tr1.a".attr === "tr2.a".attr)) + .where("a".attr > 10) + .join(tr2.where("d".attr < 100), Inner, Some("tr1.a".attr === "tr2.a".attr)) .analyze.constraints, ExpressionSet(Seq(tr1.resolveQuoted("a", caseInsensitiveResolution).get > 10, tr2.resolveQuoted("d", caseInsensitiveResolution).get < 100, @@ -224,51 +230,51 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { } test("propagating constraints in left-semi join") { - val tr1 = LocalRelation('a.int, 'b.int, 'c.int).subquery('tr1) - val tr2 = LocalRelation('a.int, 'd.int, 'e.int).subquery('tr2) + val tr1 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int).subquery("tr1") + val tr2 = LocalRelation("a".attr.int, "d".attr.int, "e".attr.int).subquery("tr2") verifyConstraints(tr1 - .where('a.attr > 10) - .join(tr2.where('d.attr < 100), LeftSemi, Some("tr1.a".attr === "tr2.a".attr)) + .where("a".attr > 10) + .join(tr2.where("d".attr < 100), LeftSemi, Some("tr1.a".attr === "tr2.a".attr)) .analyze.constraints, ExpressionSet(Seq(tr1.resolveQuoted("a", caseInsensitiveResolution).get > 10, IsNotNull(tr1.resolveQuoted("a", caseInsensitiveResolution).get)))) } test("propagating constraints in left-outer join") { - val tr1 = LocalRelation('a.int, 'b.int, 'c.int).subquery('tr1) - val tr2 = LocalRelation('a.int, 'd.int, 'e.int).subquery('tr2) + val tr1 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int).subquery("tr1") + val tr2 = LocalRelation("a".attr.int, "d".attr.int, "e".attr.int).subquery("tr2") verifyConstraints(tr1 - .where('a.attr > 10) - .join(tr2.where('d.attr < 100), LeftOuter, Some("tr1.a".attr === "tr2.a".attr)) + .where("a".attr > 10) + .join(tr2.where("d".attr < 100), LeftOuter, Some("tr1.a".attr === "tr2.a".attr)) .analyze.constraints, ExpressionSet(Seq(tr1.resolveQuoted("a", caseInsensitiveResolution).get > 10, IsNotNull(tr1.resolveQuoted("a", caseInsensitiveResolution).get)))) } test("propagating constraints in right-outer join") { - val tr1 = LocalRelation('a.int, 'b.int, 'c.int).subquery('tr1) - val tr2 = LocalRelation('a.int, 'd.int, 'e.int).subquery('tr2) + val tr1 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int).subquery("tr1") + val tr2 = LocalRelation("a".attr.int, "d".attr.int, "e".attr.int).subquery("tr2") verifyConstraints(tr1 - .where('a.attr > 10) - .join(tr2.where('d.attr < 100), RightOuter, Some("tr1.a".attr === "tr2.a".attr)) + .where("a".attr > 10) + .join(tr2.where("d".attr < 100), RightOuter, Some("tr1.a".attr === "tr2.a".attr)) .analyze.constraints, ExpressionSet(Seq(tr2.resolveQuoted("d", caseInsensitiveResolution).get < 100, IsNotNull(tr2.resolveQuoted("d", caseInsensitiveResolution).get)))) } test("propagating constraints in full-outer join") { - val tr1 = LocalRelation('a.int, 'b.int, 'c.int).subquery('tr1) - val tr2 = LocalRelation('a.int, 'd.int, 'e.int).subquery('tr2) - assert(tr1.where('a.attr > 10) - .join(tr2.where('d.attr < 100), FullOuter, Some("tr1.a".attr === "tr2.a".attr)) + val tr1 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int).subquery("tr1") + val tr2 = LocalRelation("a".attr.int, "d".attr.int, "e".attr.int).subquery("tr2") + assert(tr1.where("a".attr > 10) + .join(tr2.where("d".attr < 100), FullOuter, Some("tr1.a".attr === "tr2.a".attr)) .analyze.constraints.isEmpty) } test("infer additional constraints in filters") { - val tr = LocalRelation('a.int, 'b.int, 'c.int) + val tr = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) verifyConstraints(tr - .where('a.attr > 10 && 'a.attr === 'b.attr) + .where("a".attr > 10 && "a".attr === "b".attr) .analyze.constraints, ExpressionSet(Seq(resolveColumn(tr, "a") > 10, resolveColumn(tr, "b") > 10, @@ -278,10 +284,10 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { } test("infer constraints on cast") { - val tr = LocalRelation('a.int, 'b.long, 'c.int, 'd.long, 'e.int) + val tr = LocalRelation("a".attr.int, "b".attr.long, "c".attr.int, "d".attr.long, "e".attr.int) verifyConstraints( - tr.where('a.attr === 'b.attr && - 'c.attr + 100 > 'd.attr && + tr.where("a".attr === "b".attr && + "c".attr + 100 > "d".attr && IsNotNull(Cast(Cast(resolveColumn(tr, "e"), LongType), LongType))).analyze.constraints, ExpressionSet(Seq( castWithTimeZone(resolveColumn(tr, "a"), LongType) === resolveColumn(tr, "b"), @@ -295,9 +301,9 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { } test("infer isnotnull constraints from compound expressions") { - val tr = LocalRelation('a.int, 'b.long, 'c.int, 'd.long, 'e.int) + val tr = LocalRelation("a".attr.int, "b".attr.long, "c".attr.int, "d".attr.long, "e".attr.int) verifyConstraints( - tr.where('a.attr + 'b.attr === 'c.attr && + tr.where("a".attr + "b".attr === "c".attr && IsNotNull( Cast( Cast(Cast(resolveColumn(tr, "e"), LongType), LongType), LongType))).analyze.constraints, @@ -313,7 +319,9 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { resolveColumn(tr, "e"), LongType), LongType), LongType))))) verifyConstraints( - tr.where(('a.attr * 'b.attr + 100) === 'c.attr && 'd / 10 === 'e).analyze.constraints, + tr + .where(("a".attr * "b".attr + 100) === "c".attr && "d".attr / 10 === "e".attr) + .analyze.constraints, ExpressionSet(Seq( castWithTimeZone(resolveColumn(tr, "a"), LongType) * resolveColumn(tr, "b") + castWithTimeZone(100, LongType) === @@ -328,7 +336,9 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { IsNotNull(resolveColumn(tr, "e"))))) verifyConstraints( - tr.where(('a.attr * 'b.attr - 10) >= 'c.attr && 'd / 10 < 'e).analyze.constraints, + tr + .where(("a".attr * "b".attr - 10) >= "c".attr && "d".attr / 10 < "e".attr) + .analyze.constraints, ExpressionSet(Seq( castWithTimeZone(resolveColumn(tr, "a"), LongType) * resolveColumn(tr, "b") - castWithTimeZone(10, LongType) >= @@ -343,7 +353,7 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { IsNotNull(resolveColumn(tr, "e"))))) verifyConstraints( - tr.where('a.attr + 'b.attr - 'c.attr * 'd.attr > 'e.attr * 1000).analyze.constraints, + tr.where("a".attr + "b".attr - "c".attr * "d".attr > "e".attr * 1000).analyze.constraints, ExpressionSet(Seq( (castWithTimeZone(resolveColumn(tr, "a"), LongType) + resolveColumn(tr, "b")) - (castWithTimeZone(resolveColumn(tr, "c"), LongType) * resolveColumn(tr, "d")) > @@ -356,7 +366,7 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { // The constraint IsNotNull(IsNotNull(expr)) doesn't guarantee expr is not null. verifyConstraints( - tr.where('a.attr === 'c.attr && + tr.where("a".attr === "c".attr && IsNotNull(IsNotNull(resolveColumn(tr, "b")))).analyze.constraints, ExpressionSet(Seq( resolveColumn(tr, "a") === resolveColumn(tr, "c"), @@ -365,7 +375,7 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { IsNotNull(resolveColumn(tr, "c"))))) verifyConstraints( - tr.where('a.attr === 1 && IsNotNull(resolveColumn(tr, "b")) && + tr.where("a".attr === 1 && IsNotNull(resolveColumn(tr, "b")) && IsNotNull(resolveColumn(tr, "c"))).analyze.constraints, ExpressionSet(Seq( resolveColumn(tr, "a") === 1, @@ -375,7 +385,7 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { } test("infer IsNotNull constraints from non-nullable attributes") { - val tr = LocalRelation('a.int, AttributeReference("b", IntegerType, nullable = false)(), + val tr = LocalRelation("a".attr.int, AttributeReference("b", IntegerType, nullable = false)(), AttributeReference("c", StringType, nullable = false)()) verifyConstraints(tr.analyze.constraints, @@ -383,16 +393,16 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { } test("not infer non-deterministic constraints") { - val tr = LocalRelation('a.int, 'b.string, 'c.int) + val tr = LocalRelation("a".attr.int, "b".attr.string, "c".attr.int) verifyConstraints(tr - .where('a.attr === Rand(0)) + .where("a".attr === Rand(0)) .analyze.constraints, ExpressionSet(Seq(IsNotNull(resolveColumn(tr, "a"))))) verifyConstraints(tr - .where('a.attr === InputFileName()) - .where('a.attr =!= 'c.attr) + .where("a".attr === InputFileName()) + .where("a".attr =!= "c".attr) .analyze.constraints, ExpressionSet(Seq(resolveColumn(tr, "a") =!= resolveColumn(tr, "c"), IsNotNull(resolveColumn(tr, "a")), @@ -400,8 +410,8 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { } test("enable/disable constraint propagation") { - val tr = LocalRelation('a.int, 'b.string, 'c.int) - val filterRelation = tr.where('a.attr > 10) + val tr = LocalRelation("a".attr.int, "b".attr.string, "c".attr.int) + val filterRelation = tr.where("a".attr > 10) withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "true") { assert(filterRelation.analyze.constraints.nonEmpty) @@ -411,8 +421,9 @@ class ConstraintPropagationSuite extends SparkFunSuite with PlanTest { assert(filterRelation.analyze.constraints.isEmpty) } - val aliasedRelation = tr.where('c.attr > 10 && 'a.attr < 5) - .groupBy('a, 'c, 'b)('a, 'c.as("c1"), count('a).as("a3")).select('c1, 'a, 'a3) + val aliasedRelation = tr.where("c".attr > 10 && "a".attr < 5) + .groupBy("a".attr, "c".attr, "b".attr)("a".attr, "c".attr.as("c1"), count("a".attr).as("a3")) + .select("c1".attr, "a".attr, "a3".attr) withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "true") { assert(aliasedRelation.analyze.constraints.nonEmpty) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/QueryPlanSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/QueryPlanSuite.scala index 404c8895c4d1..e49b578ea406 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/QueryPlanSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/QueryPlanSuite.scala @@ -97,7 +97,7 @@ class QueryPlanSuite extends SparkFunSuite { } } - val t = LocalRelation('a.int, 'b.int) + val t = LocalRelation("a".attr.int, "b".attr.int) val plan = t.select($"a", $"b").select($"a", $"b").select($"a", $"b").analyze assert(testRule(plan).resolved) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala index fbaaf807af5d..30e29d245326 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala @@ -29,8 +29,8 @@ import org.apache.spark.sql.catalyst.util._ * Tests for the sameResult function of [[LogicalPlan]]. */ class SameResultSuite extends SparkFunSuite { - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) - val testRelation2 = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) + val testRelation2 = LocalRelation("a".attr.int, "b".attr.int, "c".attr.int) object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("EliminateResolvedHint", Once, EliminateResolvedHint) :: Nil @@ -51,21 +51,24 @@ class SameResultSuite extends SparkFunSuite { } test("projections") { - assertSameResult(testRelation.select('a), testRelation2.select('a)) - assertSameResult(testRelation.select('b), testRelation2.select('b)) - assertSameResult(testRelation.select('a, 'b), testRelation2.select('a, 'b)) - assertSameResult(testRelation.select('b, 'a), testRelation2.select('b, 'a)) + assertSameResult(testRelation.select("a".attr), testRelation2.select("a".attr)) + assertSameResult(testRelation.select("b".attr), testRelation2.select("b".attr)) + assertSameResult(testRelation.select("a".attr, "b".attr), + testRelation2.select("a".attr, "b".attr)) + assertSameResult(testRelation.select("b".attr, "a".attr), + testRelation2.select("b".attr, "a".attr)) - assertSameResult(testRelation, testRelation2.select('a), result = false) - assertSameResult(testRelation.select('b, 'a), testRelation2.select('a, 'b), result = false) + assertSameResult(testRelation, testRelation2.select("a".attr), result = false) + assertSameResult(testRelation.select("b".attr, "a".attr), + testRelation2.select("a".attr, "b".attr), result = false) } test("filters") { - assertSameResult(testRelation.where('a === 'b), testRelation2.where('a === 'b)) + assertSameResult(testRelation.where('a === "b".attr), testRelation2.where('a === "b".attr)) } test("sorts") { - assertSameResult(testRelation.orderBy('a.asc), testRelation2.orderBy('a.asc)) + assertSameResult(testRelation.orderBy("a".attr.asc), testRelation2.orderBy("a".attr.asc)) } test("union") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanIntegritySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanIntegritySuite.scala index 6f342b8d9437..e877cf5ee5a5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanIntegritySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanIntegritySuite.scala @@ -31,7 +31,7 @@ class LogicalPlanIntegritySuite extends PlanTest { } test("Checks if the same `ExprId` refers to a semantically-equal attribute in a plan output") { - val t = LocalRelation('a.int, 'b.int) + val t = LocalRelation("a".attr.int, "b".attr.int) assert(hasUniqueExprIdsForOutput(OutputTestPlan(t, t.output))) assert(!hasUniqueExprIdsForOutput(OutputTestPlan(t, t.output.zipWithIndex.map { case (a, i) => AttributeReference(s"c$i", LongType)(a.exprId) @@ -39,7 +39,7 @@ class LogicalPlanIntegritySuite extends PlanTest { } test("Checks if reference ExprIds are not reused when assigning a new ExprId") { - val t = LocalRelation('a.int, 'b.int) + val t = LocalRelation("a".attr.int, "b".attr.int) val Seq(a, b) = t.output assert(checkIfSameExprIdNotReused(t.select(Alias(a + 1, "a")()))) assert(!checkIfSameExprIdNotReused(t.select(Alias(a + 1, "a")(exprId = a.exprId)))) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala index 749fed394073..67c7221f0091 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala @@ -52,7 +52,7 @@ class BasicStatsEstimationSuite extends PlanTest with StatsEstimationTestBase { } test("windows") { - val windows = plan.window(Seq(min(attribute).as('sum_attr)), Seq(attribute), Nil) + val windows = plan.window(Seq(min(attribute).as("sum_attr")), Seq(attribute), Nil) val windowsStats = Statistics(sizeInBytes = plan.size.get * (4 + 4 + 8) / (4 + 8)) checkStats( windows, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/AggregatingAccumulatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/AggregatingAccumulatorSuite.scala index a33b9fad7ff4..9a4ddb4360f6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/AggregatingAccumulatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/AggregatingAccumulatorSuite.scala @@ -35,9 +35,9 @@ class AggregatingAccumulatorSuite extends SparkFunSuite with SharedSparkSession with ExpressionEvalHelper { - private val a = 'a.long - private val b = 'b.string - private val c = 'c.double + private val a = "a".attr.long + private val b = "b".attr.string + private val c = "c".attr.double private val inputAttributes = Seq(a, b, c) private def str(s: String): UTF8String = UTF8String.fromString(s) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/CoGroupedIteratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/CoGroupedIteratorSuite.scala index 4ff96e6574ca..f93161f831dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/CoGroupedIteratorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/CoGroupedIteratorSuite.scala @@ -26,9 +26,11 @@ class CoGroupedIteratorSuite extends SparkFunSuite with ExpressionEvalHelper { test("basic") { val leftInput = Seq(create_row(1, "a"), create_row(1, "b"), create_row(2, "c")).iterator val rightInput = Seq(create_row(1, 2L), create_row(2, 3L), create_row(3, 4L)).iterator - val leftGrouped = GroupedIterator(leftInput, Seq('i.int.at(0)), Seq('i.int, 's.string)) - val rightGrouped = GroupedIterator(rightInput, Seq('i.int.at(0)), Seq('i.int, 'l.long)) - val cogrouped = new CoGroupedIterator(leftGrouped, rightGrouped, Seq('i.int)) + val leftGrouped = + GroupedIterator(leftInput, Seq("i".attr.int.at(0)), Seq("i".attr.int, "s".attr.string)) + val rightGrouped = + GroupedIterator(rightInput, Seq("i".attr.int.at(0)), Seq("i".attr.int, "l".attr.long)) + val cogrouped = new CoGroupedIterator(leftGrouped, rightGrouped, Seq("i".attr.int)) val result = cogrouped.map { case (key, leftData, rightData) => @@ -52,9 +54,11 @@ class CoGroupedIteratorSuite extends SparkFunSuite with ExpressionEvalHelper { test("SPARK-11393: respect the fact that GroupedIterator.hasNext is not idempotent") { val leftInput = Seq(create_row(2, "a")).iterator val rightInput = Seq(create_row(1, 2L)).iterator - val leftGrouped = GroupedIterator(leftInput, Seq('i.int.at(0)), Seq('i.int, 's.string)) - val rightGrouped = GroupedIterator(rightInput, Seq('i.int.at(0)), Seq('i.int, 'l.long)) - val cogrouped = new CoGroupedIterator(leftGrouped, rightGrouped, Seq('i.int)) + val leftGrouped = + GroupedIterator(leftInput, Seq("i".attr.int.at(0)), Seq("i".attr.int, "s".attr.string)) + val rightGrouped = + GroupedIterator(rightInput, Seq("i".attr.int.at(0)), Seq("i".attr.int, "l".attr.long)) + val cogrouped = new CoGroupedIterator(leftGrouped, rightGrouped, Seq("i".attr.int)) val result = cogrouped.map { case (key, leftData, rightData) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/GroupedIteratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/GroupedIteratorSuite.scala index 4b2a2b439c89..9f90012fa346 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/GroupedIteratorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/GroupedIteratorSuite.scala @@ -32,7 +32,7 @@ class GroupedIteratorSuite extends SparkFunSuite { val fromRow = encoder.createDeserializer() val input = Seq(Row(1, "a"), Row(1, "b"), Row(2, "c")) val grouped = GroupedIterator(input.iterator.map(toRow), - Seq('i.int.at(0)), schema.toAttributes) + Seq("i".attr.int.at(0)), schema.toAttributes) val result = grouped.map { case (key, data) => @@ -59,7 +59,7 @@ class GroupedIteratorSuite extends SparkFunSuite { Row(3, 2L, "e")) val grouped = GroupedIterator(input.iterator.map(toRow), - Seq('i.int.at(0), 'l.long.at(1)), schema.toAttributes) + Seq("i".attr.int.at(0), "l".attr.long.at(1)), schema.toAttributes) val result = grouped.map { case (key, data) => @@ -80,7 +80,7 @@ class GroupedIteratorSuite extends SparkFunSuite { val toRow = encoder.createSerializer() val input = Seq(Row(1, "a"), Row(1, "b"), Row(2, "c")) val grouped = GroupedIterator(input.iterator.map(toRow), - Seq('i.int.at(0)), schema.toAttributes) + Seq("i".attr.int.at(0)), schema.toAttributes) assert(grouped.length == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala index 6a4f3f62641f..2509f7ef0f92 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala @@ -43,13 +43,15 @@ class SortSuite extends SparkPlanTest with SharedSparkSession { checkAnswer( input.toDF("a", "b", "c"), - (child: SparkPlan) => SortExec('a.asc :: 'b.asc :: Nil, global = true, child = child), + (child: SparkPlan) => SortExec("a".attr.asc :: "b".attr.asc :: Nil, + global = true, child = child), input.sortBy(t => (t._1, t._2)).map(Row.fromTuple), sortAnswers = false) checkAnswer( input.toDF("a", "b", "c"), - (child: SparkPlan) => SortExec('b.asc :: 'a.asc :: Nil, global = true, child = child), + (child: SparkPlan) => SortExec("b".attr.asc :: "a".attr.asc :: Nil, + global = true, child = child), input.sortBy(t => (t._2, t._1)).map(Row.fromTuple), sortAnswers = false) } @@ -58,9 +60,9 @@ class SortSuite extends SparkPlanTest with SharedSparkSession { checkThatPlansAgree( (1 to 100).map(v => Tuple1(v)).toDF().selectExpr("NULL as a"), (child: SparkPlan) => - GlobalLimitExec(10, SortExec('a.asc :: Nil, global = true, child = child)), + GlobalLimitExec(10, SortExec("a".attr.asc :: Nil, global = true, child = child)), (child: SparkPlan) => - GlobalLimitExec(10, ReferenceSort('a.asc :: Nil, global = true, child)), + GlobalLimitExec(10, ReferenceSort("a".attr.asc :: Nil, global = true, child)), sortAnswers = false ) } @@ -69,15 +71,15 @@ class SortSuite extends SparkPlanTest with SharedSparkSession { checkThatPlansAgree( (1 to 100).map(v => Tuple1(v)).toDF("a"), (child: SparkPlan) => - GlobalLimitExec(10, SortExec('a.asc :: Nil, global = true, child = child)), + GlobalLimitExec(10, SortExec("a".attr.asc :: Nil, global = true, child = child)), (child: SparkPlan) => - GlobalLimitExec(10, ReferenceSort('a.asc :: Nil, global = true, child)), + GlobalLimitExec(10, ReferenceSort("a".attr.asc :: Nil, global = true, child)), sortAnswers = false ) } test("sorting does not crash for large inputs") { - val sortOrder = 'a.asc :: Nil + val sortOrder = "a".attr.asc :: Nil val stringLength = 1024 * 1024 * 2 checkThatPlansAgree( Seq(Tuple1("a" * stringLength), Tuple1("b" * stringLength)).toDF("a").repartition(1), @@ -91,8 +93,8 @@ class SortSuite extends SparkPlanTest with SharedSparkSession { AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "unsafe external sort") { checkThatPlansAgree( (1 to 100).map(v => Tuple1(v)).toDF("a"), - (child: SparkPlan) => SortExec('a.asc :: Nil, global = true, child = child), - (child: SparkPlan) => ReferenceSort('a.asc :: Nil, global = true, child), + (child: SparkPlan) => SortExec("a".attr.asc :: Nil, global = true, child = child), + (child: SparkPlan) => ReferenceSort("a".attr.asc :: Nil, global = true, child), sortAnswers = false) } } @@ -105,7 +107,8 @@ class SortSuite extends SparkPlanTest with SharedSparkSession { ) checkAnswer( input.toDF("a", "b", "c"), - (child: SparkPlan) => SortExec(Stream('a.asc, 'b.asc, 'c.asc), global = true, child = child), + (child: SparkPlan) => SortExec(Stream("a".attr.asc, "b".attr.asc, "c".attr.asc), + global = true, child = child), input.sortBy(t => (t._1, t._2, t._3)).map(Row.fromTuple), sortAnswers = false) } @@ -115,7 +118,8 @@ class SortSuite extends SparkPlanTest with SharedSparkSession { dataType <- DataTypeTestUtils.atomicTypes ++ Set(NullType); nullable <- Seq(true, false); sortOrder <- - Seq('a.asc :: Nil, 'a.asc_nullsLast :: Nil, 'a.desc :: Nil, 'a.desc_nullsFirst :: Nil); + Seq("a".attr.asc :: Nil, "a".attr.asc_nullsLast :: Nil, "a".attr.desc :: Nil, + "a".attr.desc_nullsFirst :: Nil); randomDataGenerator <- RandomDataGenerator.forType(dataType, nullable) ) { test(s"sorting on $dataType with nullable=$nullable, sortOrder=$sortOrder") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index f1788e9c31af..a4d71dfd7ee1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -281,7 +281,7 @@ class SparkSqlParserSuite extends AnalysisTest { |FROM testData """.stripMargin, ScriptTransformation( - Seq('a, 'b, 'c), + Seq("a".attr, "b".attr, "c".attr), "cat", Seq(AttributeReference("a", StringType)(), AttributeReference("b", StringType)(), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala index 6ec5c6287eed..b3ce03f1554b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala @@ -58,7 +58,7 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSparkSession { private def noOpFilter(plan: SparkPlan): SparkPlan = FilterExec(Literal(true), plan) val limit = 250 - val sortOrder = 'a.desc :: 'b.desc :: Nil + val sortOrder = "a".attr.desc :: "b".attr.desc :: Nil test("TakeOrderedAndProject.doExecute without project") { withClue(s"seed = $seed") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 96f9421e1d98..6d51d67b0296 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -284,11 +284,12 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { "func", Seq.empty, plans.table("e"), null) compareTransformQuery("select transform(a, b) using 'func' from e where f < 10", - p.copy(child = p.child.where('f < 10), output = Seq('key.string, 'value.string))) + p.copy(child = p.child.where("f".attr < 10), + output = Seq("key".attr.string, "value".attr.string))) compareTransformQuery("map a, b using 'func' as c, d from e", - p.copy(output = Seq('c.string, 'd.string))) + p.copy(output = Seq("c".attr.string, "d".attr.string))) compareTransformQuery("reduce a, b using 'func' as (c int, d decimal(10, 0)) from e", - p.copy(output = Seq('c.int, 'd.decimal(10, 0)))) + p.copy(output = Seq("c".attr.int, "d".attr.decimal(10, 0)))) } test("use backticks in output of Script Transform") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala index b94918eccd46..6901fe97b185 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala @@ -26,21 +26,21 @@ import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructT class DataSourceStrategySuite extends PlanTest with SharedSparkSession { val attrInts = Seq( - 'cint.int, - Symbol("c.int").int, - GetStructField('a.struct(StructType( + "cint".attr.int, + "c.int".attr.int, + GetStructField("a".attr.struct(StructType( StructField("cstr", StringType, nullable = true) :: StructField("cint", IntegerType, nullable = true) :: Nil)), 1, None), - GetStructField('a.struct(StructType( + GetStructField("a".attr.struct(StructType( StructField("c.int", IntegerType, nullable = true) :: StructField("cstr", StringType, nullable = true) :: Nil)), 0, None), - GetStructField(Symbol("a.b").struct(StructType( + GetStructField("a.b".attr.struct(StructType( StructField("cstr1", StringType, nullable = true) :: StructField("cstr2", StringType, nullable = true) :: StructField("cint", IntegerType, nullable = true) :: Nil)), 2, None), - GetStructField(Symbol("a.b").struct(StructType( + GetStructField("a.b".attr.struct(StructType( StructField("c.int", IntegerType, nullable = true) :: Nil)), 0, None), - GetStructField(GetStructField('a.struct(StructType( + GetStructField(GetStructField("a".attr.struct(StructType( StructField("cstr1", StringType, nullable = true) :: StructField("b", StructType(StructField("cint", IntegerType, nullable = true) :: StructField("cstr2", StringType, nullable = true) :: Nil)) :: Nil)), 1, None), 0, None) @@ -55,21 +55,21 @@ class DataSourceStrategySuite extends PlanTest with SharedSparkSession { )) val attrStrs = Seq( - 'cstr.string, - Symbol("c.str").string, - GetStructField('a.struct(StructType( + "cstr".attr.string, + "c.str".attr.string, + GetStructField("a".attr.struct(StructType( StructField("cint", IntegerType, nullable = true) :: StructField("cstr", StringType, nullable = true) :: Nil)), 1, None), - GetStructField('a.struct(StructType( + GetStructField("a".attr.struct(StructType( StructField("c.str", StringType, nullable = true) :: StructField("cint", IntegerType, nullable = true) :: Nil)), 0, None), - GetStructField(Symbol("a.b").struct(StructType( + GetStructField("a.b".attr.struct(StructType( StructField("cint1", IntegerType, nullable = true) :: StructField("cint2", IntegerType, nullable = true) :: StructField("cstr", StringType, nullable = true) :: Nil)), 2, None), - GetStructField(Symbol("a.b").struct(StructType( + GetStructField("a.b".attr.struct(StructType( StructField("c.str", StringType, nullable = true) :: Nil)), 0, None), - GetStructField(GetStructField('a.struct(StructType( + GetStructField(GetStructField("a".attr.struct(StructType( StructField("cint1", IntegerType, nullable = true) :: StructField("b", StructType(StructField("cstr", StringType, nullable = true) :: StructField("cint2", IntegerType, nullable = true) :: Nil)) :: Nil)), 1, None), 0, None) @@ -280,7 +280,7 @@ class DataSourceStrategySuite extends PlanTest with SharedSparkSession { }} test("SPARK-26865 DataSourceV2Strategy should push normalized filters") { - val attrInt = 'cint.int + val attrInt = "cint".attr.int assertResult(Seq(IsNotNull(attrInt))) { DataSourceStrategy.normalizeExprs(Seq(IsNotNull(attrInt.withName("CiNt"))), Seq(attrInt)) } @@ -308,7 +308,7 @@ class DataSourceStrategySuite extends PlanTest with SharedSparkSession { } // `Abs(col)` can not be pushed down, so it returns `None` - assert(PushableColumnAndNestedColumn.unapply(Abs('col.int)) === None) + assert(PushableColumnAndNestedColumn.unapply(Abs("col".attr.int)) === None) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index c90732183cb7..9aec75de194e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -532,7 +532,7 @@ abstract class SchemaPruningSuite Seq(Concat(Seq($"name.first", $"name.last")), Concat(Seq($"name.last", $"name.first"))) ), - Seq('a.string, 'b.string), + Seq("a".attr.string, "b".attr.string), sql("select * from contacts").logicalPlan ).toDF() checkScan(query1, "struct>") @@ -549,7 +549,7 @@ abstract class SchemaPruningSuite val name = StructType.fromDDL("first string, middle string, last string") val query2 = Expand( Seq(Seq($"name", $"name.last")), - Seq('a.struct(name), 'b.string), + Seq("a".attr.struct(name), "b".attr.string), sql("select * from contacts").logicalPlan ).toDF() checkScan(query2, "struct>") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 329a3e498379..9f018839f74a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -1263,7 +1263,7 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } withSQLConf( - // Makes sure disabling 'spark.sql.parquet.recordFilter' still enables + // Makes sure disabling "spark".attr.sql.parquet.recordFilter' still enables // row group level filtering. SQLConf.PARQUET_RECORD_FILTER_ENABLED.key -> "false", SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") { @@ -1290,7 +1290,7 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared import testImplicits._ withSQLConf( - // Makes sure disabling 'spark.sql.parquet.recordFilter' still enables + // Makes sure disabling "spark".attr.sql.parquet.recordFilter' still enables // row group level filtering. SQLConf.PARQUET_RECORD_FILTER_ENABLED.key -> "false", SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", @@ -1327,39 +1327,39 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared test("filter pushdown - StringStartsWith") { withParquetDataFrame((1 to 4).map(i => Tuple1(i + "str" + i))) { implicit df => checkFilterPredicate( - '_1.startsWith("").asInstanceOf[Predicate], + "_1".attr.startsWith("").asInstanceOf[Predicate], classOf[UserDefinedByInstance[_, _]], Seq("1str1", "2str2", "3str3", "4str4").map(Row(_))) Seq("2", "2s", "2st", "2str", "2str2").foreach { prefix => checkFilterPredicate( - '_1.startsWith(prefix).asInstanceOf[Predicate], + "_1".attr.startsWith(prefix).asInstanceOf[Predicate], classOf[UserDefinedByInstance[_, _]], "2str2") } Seq("2S", "null", "2str22").foreach { prefix => checkFilterPredicate( - '_1.startsWith(prefix).asInstanceOf[Predicate], + "_1".attr.startsWith(prefix).asInstanceOf[Predicate], classOf[UserDefinedByInstance[_, _]], Seq.empty[Row]) } checkFilterPredicate( - !'_1.startsWith("").asInstanceOf[Predicate], + !"_1".attr.startsWith("").asInstanceOf[Predicate], classOf[Operators.Not], Seq().map(Row(_))) Seq("2", "2s", "2st", "2str", "2str2").foreach { prefix => checkFilterPredicate( - !'_1.startsWith(prefix).asInstanceOf[Predicate], + !"_1".attr.startsWith(prefix).asInstanceOf[Predicate], classOf[Operators.Not], Seq("1str1", "3str3", "4str4").map(Row(_))) } Seq("2S", "null", "2str22").foreach { prefix => checkFilterPredicate( - !'_1.startsWith(prefix).asInstanceOf[Predicate], + !"_1".attr.startsWith(prefix).asInstanceOf[Predicate], classOf[Operators.Not], Seq("1str1", "2str2", "3str3", "4str4").map(Row(_))) } @@ -1373,7 +1373,7 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared // SPARK-28371: make sure filter is null-safe. withParquetDataFrame(Seq(Tuple1[String](null))) { implicit df => checkFilterPredicate( - '_1.startsWith("blah").asInstanceOf[Predicate], + "_1".attr.startsWith("blah").asInstanceOf[Predicate], classOf[UserDefinedByInstance[_, _]], Seq.empty[Row]) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala index 81ce979ef0b6..5ca53220b43d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala @@ -36,7 +36,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll with override def beforeAll(): Unit = { super.beforeAll() - targetAttributes = Seq('a.int, 'd.int, 'b.int, 'c.int) + targetAttributes = Seq("a".attr.int, "d".attr.int, "b".attr.int, "c".attr.int) targetPartitionSchema = new StructType() .add("b", IntegerType) .add("c", IntegerType) @@ -74,7 +74,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll with caseSensitive) { intercept[AssertionError] { rule.convertStaticPartitions( - sourceAttributes = Seq('e.int, 'f.int), + sourceAttributes = Seq("e".attr.int, "f".attr.int), providedPartitions = Map("b" -> None, "c" -> None), targetAttributes = targetAttributes, targetPartitionSchema = targetPartitionSchema) @@ -85,7 +85,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll with // Missing columns. intercept[AnalysisException] { rule.convertStaticPartitions( - sourceAttributes = Seq('e.int), + sourceAttributes = Seq("e".attr.int), providedPartitions = Map("b" -> Some("1"), "c" -> None), targetAttributes = targetAttributes, targetPartitionSchema = targetPartitionSchema) @@ -96,7 +96,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll with // Missing partitioning columns. intercept[AnalysisException] { rule.convertStaticPartitions( - sourceAttributes = Seq('e.int, 'f.int), + sourceAttributes = Seq("e".attr.int, "f".attr.int), providedPartitions = Map("b" -> Some("1")), targetAttributes = targetAttributes, targetPartitionSchema = targetPartitionSchema) @@ -105,7 +105,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll with // Missing partitioning columns. intercept[AnalysisException] { rule.convertStaticPartitions( - sourceAttributes = Seq('e.int, 'f.int, 'g.int), + sourceAttributes = Seq("e".attr.int, "f".attr.int, "g".attr.int), providedPartitions = Map("b" -> Some("1")), targetAttributes = targetAttributes, targetPartitionSchema = targetPartitionSchema) @@ -114,7 +114,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll with // Wrong partitioning columns. intercept[AnalysisException] { rule.convertStaticPartitions( - sourceAttributes = Seq('e.int, 'f.int), + sourceAttributes = Seq("e".attr.int, "f".attr.int), providedPartitions = Map("b" -> Some("1"), "d" -> None), targetAttributes = targetAttributes, targetPartitionSchema = targetPartitionSchema) @@ -125,7 +125,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll with // Wrong partitioning columns. intercept[AnalysisException] { rule.convertStaticPartitions( - sourceAttributes = Seq('e.int, 'f.int), + sourceAttributes = Seq("e".attr.int, "f".attr.int), providedPartitions = Map("b" -> Some("1"), "d" -> Some("2")), targetAttributes = targetAttributes, targetPartitionSchema = targetPartitionSchema) @@ -134,7 +134,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll with // Wrong partitioning columns. intercept[AnalysisException] { rule.convertStaticPartitions( - sourceAttributes = Seq('e.int), + sourceAttributes = Seq("e".attr.int), providedPartitions = Map("b" -> Some("1"), "c" -> Some("3"), "d" -> Some("2")), targetAttributes = targetAttributes, targetPartitionSchema = targetPartitionSchema) @@ -144,7 +144,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll with // Wrong partitioning columns. intercept[AnalysisException] { rule.convertStaticPartitions( - sourceAttributes = Seq('e.int, 'f.int), + sourceAttributes = Seq("e".attr.int, "f".attr.int), providedPartitions = Map("b" -> Some("1"), "C" -> Some("3")), targetAttributes = targetAttributes, targetPartitionSchema = targetPartitionSchema) @@ -156,7 +156,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll with // Static partitions need to appear before dynamic partitions. intercept[AnalysisException] { rule.convertStaticPartitions( - sourceAttributes = Seq('e.int, 'f.int), + sourceAttributes = Seq("e".attr.int, "f".attr.int), providedPartitions = Map("b" -> None, "c" -> Some("3")), targetAttributes = targetAttributes, targetPartitionSchema = targetPartitionSchema) @@ -165,7 +165,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll with testRule("All static partitions", caseSensitive) { if (!caseSensitive) { - val nonPartitionedAttributes = Seq('e.int, 'f.int) + val nonPartitionedAttributes = Seq("e".attr.int, "f".attr.int) val expected = nonPartitionedAttributes ++ Seq(cast(Literal("1"), IntegerType), cast(Literal("3"), IntegerType)) val actual = rule.convertStaticPartitions( @@ -177,7 +177,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll with } { - val nonPartitionedAttributes = Seq('e.int, 'f.int) + val nonPartitionedAttributes = Seq("e".attr.int, "f".attr.int) val expected = nonPartitionedAttributes ++ Seq(cast(Literal("1"), IntegerType), cast(Literal("3"), IntegerType)) val actual = rule.convertStaticPartitions( @@ -190,20 +190,20 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll with // Test the case having a single static partition column. { - val nonPartitionedAttributes = Seq('e.int, 'f.int) + val nonPartitionedAttributes = Seq("e".attr.int, "f".attr.int) val expected = nonPartitionedAttributes ++ Seq(cast(Literal("1"), IntegerType)) val actual = rule.convertStaticPartitions( sourceAttributes = nonPartitionedAttributes, providedPartitions = Map("b" -> Some("1")), - targetAttributes = Seq('a.int, 'd.int, 'b.int), + targetAttributes = Seq("a".attr.int, "d".attr.int, "b".attr.int), targetPartitionSchema = new StructType().add("b", IntegerType)) checkProjectList(actual, expected) } } testRule("Static partition and dynamic partition", caseSensitive) { - val nonPartitionedAttributes = Seq('e.int, 'f.int) - val dynamicPartitionAttributes = Seq('g.int) + val nonPartitionedAttributes = Seq("e".attr.int, "f".attr.int) + val dynamicPartitionAttributes = Seq("g".attr.int) val expected = nonPartitionedAttributes ++ Seq(cast(Literal("1"), IntegerType)) ++ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala index ab37645b1c90..db66cfdb06c7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala @@ -65,8 +65,8 @@ class PruneFileSourcePartitionsSuite extends PrunePartitionSuiteBase { options = Map.empty)(sparkSession = spark) val logicalRelation = LogicalRelation(relation, tableMeta) - val query = Project(Seq(Symbol("i"), Symbol("p")), - Filter(Symbol("p") === 1, logicalRelation)).analyze + val query = Project(Seq("i".attr, "p".attr), + Filter("p".attr === 1, logicalRelation)).analyze val optimized = Optimize.execute(query) assert(optimized.missingInput.isEmpty)