Skip to content

Commit

Permalink
Fix bug of Constraint
Browse files Browse the repository at this point in the history
  • Loading branch information
gengliangwang committed Jan 16, 2018
1 parent 7bcc266 commit 8a22e1d
Show file tree
Hide file tree
Showing 2 changed files with 19 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -96,10 +96,15 @@ trait QueryPlanConstraints { self: LogicalPlan =>

// Collect aliases from expressions of the whole tree rooted by the current QueryPlan node, so
// we may avoid producing recursive constraints.
private lazy val aliasMap: AttributeMap[Expression] = AttributeMap(
expressions.collect {
private lazy val aliasMap: AttributeMap[Expression] = {
val aliases = expressions.collect {
case a: Alias if !a.child.isInstanceOf[Literal] => (a.toAttribute, a.child)
} ++ children.flatMap(_.asInstanceOf[QueryPlanConstraints].aliasMap))
} ++ children.flatMap(_.asInstanceOf[QueryPlanConstraints].aliasMap)
AttributeMap(aliases.filter {
case (_, child) => child.references.nonEmpty && child.references.subsetOf(outputSet)
})
}

// Note: the explicit cast is necessary, since Scala compiler fails to infer the type.

/**
Expand Down
11 changes: 11 additions & 0 deletions sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -2717,6 +2717,17 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
}
}

test("SPARK-23079: constraints should be inferred correctly with aliases") {
withTable("t") {
spark.range(5).write.saveAsTable("t")
val t = spark.read.table("t")
val left = t.withColumn("xid", $"id" + lit(1)).as("x")
val right = t.withColumnRenamed("id", "xid").as("y")
val df = left.join(right, "xid").filter("id = 3").toDF()
checkAnswer(df, Row(4, 3))
}
}

test("SRARK-22266: the same aggregate function was calculated multiple times") {
val query = "SELECT a, max(b+1), max(b+1) + 1 FROM testData2 GROUP BY a"
val df = sql(query)
Expand Down

0 comments on commit 8a22e1d

Please sign in to comment.