-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-17154][SQL] Wrong result can be returned or AnalysisException can be thrown after self-join or similar operations #14719
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
05872b7
91cb915
dd0ddbc
74eb4aa
48a0775
9ddc9d8
148b6d5
021977f
b09c0d7
ccf71fc
d458b79
437ac99
9ad2c85
929f2a8
15bf529
e91a24e
5d1ff3e
a3f32c4
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -556,31 +556,31 @@ class Analyzer( | |
| right.collect { | ||
| // Handle base relations that might appear more than once. | ||
| case oldVersion: MultiInstanceRelation | ||
| if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => | ||
| if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => | ||
| val newVersion = oldVersion.newInstance() | ||
| (oldVersion, newVersion) | ||
|
|
||
| case oldVersion: SerializeFromObject | ||
| if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => | ||
| if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => | ||
| (oldVersion, oldVersion.copy(serializer = oldVersion.serializer.map(_.newInstance()))) | ||
|
|
||
| // Handle projects that create conflicting aliases. | ||
| case oldVersion @ Project(projectList, _) | ||
| if findAliases(projectList).intersect(conflictingAttributes).nonEmpty => | ||
| if findAliases(projectList).intersect(conflictingAttributes).nonEmpty => | ||
| (oldVersion, oldVersion.copy(projectList = newAliases(projectList))) | ||
|
|
||
| case oldVersion @ Aggregate(_, aggregateExpressions, _) | ||
| if findAliases(aggregateExpressions).intersect(conflictingAttributes).nonEmpty => | ||
| if findAliases(aggregateExpressions).intersect(conflictingAttributes).nonEmpty => | ||
| (oldVersion, oldVersion.copy(aggregateExpressions = newAliases(aggregateExpressions))) | ||
|
|
||
| case oldVersion: Generate | ||
| if oldVersion.generatedSet.intersect(conflictingAttributes).nonEmpty => | ||
| if oldVersion.generatedSet.intersect(conflictingAttributes).nonEmpty => | ||
| val newOutput = oldVersion.generatorOutput.map(_.newInstance()) | ||
| (oldVersion, oldVersion.copy(generatorOutput = newOutput)) | ||
|
|
||
| case oldVersion @ Window(windowExpressions, _, _, child) | ||
| if AttributeSet(windowExpressions.map(_.toAttribute)).intersect(conflictingAttributes) | ||
| .nonEmpty => | ||
| if AttributeSet(windowExpressions.map(_.toAttribute)).intersect(conflictingAttributes) | ||
| .nonEmpty => | ||
| (oldVersion, oldVersion.copy(windowExpressions = newAliases(windowExpressions))) | ||
| } | ||
| // Only handle first case, others will be fixed on the next pass. | ||
|
|
@@ -597,11 +597,16 @@ class Analyzer( | |
| val newRight = right transformUp { | ||
| case r if r == oldRelation => newRelation | ||
| } transformUp { | ||
| case other => other transformExpressions { | ||
| case a: Attribute => | ||
| attributeRewrites.get(a).getOrElse(a).withQualifier(a.qualifier) | ||
| } | ||
| case other => | ||
| val transformed = other transformExpressions { | ||
| case a: Attribute => | ||
| attributeRewrites.get(a).getOrElse(a).withQualifier(a.qualifier) | ||
| } | ||
|
|
||
| transformed.setPlanId(other.planId) | ||
| transformed | ||
| } | ||
| newRight.setPlanId(right.planId) | ||
| newRight | ||
| } | ||
| } | ||
|
|
@@ -664,11 +669,18 @@ class Analyzer( | |
|
|
||
| case q: LogicalPlan => | ||
| logTrace(s"Attempting to resolve ${q.simpleString}") | ||
| q transformExpressionsUp { | ||
| case u @ UnresolvedAttribute(nameParts) => | ||
| q transformExpressionsUp { | ||
| case u @ UnresolvedAttribute(nameParts, targetPlanIdOpt) => | ||
| // Leave unchanged if resolution fails. Hopefully will be resolved next round. | ||
| val result = | ||
| withPosition(u) { q.resolveChildren(nameParts, resolver).getOrElse(u) } | ||
| withPosition(u) { | ||
| targetPlanIdOpt match { | ||
| case Some(targetPlanId) => | ||
| resolveExpressionFromSpecificLogicalPlan(nameParts, q, targetPlanId) | ||
| case None => | ||
| q.resolveChildren(nameParts, resolver).getOrElse(u) | ||
| } | ||
| } | ||
| logDebug(s"Resolving $u to $result") | ||
| result | ||
| case UnresolvedExtractValue(child, fieldExpr) if child.resolved => | ||
|
|
@@ -746,6 +758,19 @@ class Analyzer( | |
| exprs.exists(_.find(_.isInstanceOf[UnresolvedDeserializer]).isDefined) | ||
| } | ||
|
|
||
| private[sql] def resolveExpressionFromSpecificLogicalPlan( | ||
| nameParts: Seq[String], | ||
| planToSearchFrom: LogicalPlan, | ||
| targetPlanId: Long): Expression = { | ||
| lazy val name = UnresolvedAttribute(nameParts).name | ||
| planToSearchFrom.findByBreadthFirst(_.planId == targetPlanId) match { | ||
| case Some(foundPlan) => | ||
| foundPlan.resolve(nameParts, resolver).get | ||
| case None => | ||
| failAnalysis(s"Could not find $name in any logical plan.") | ||
| } | ||
| } | ||
|
|
||
| protected[sql] def resolveExpression( | ||
| expr: Expression, | ||
| plan: LogicalPlan, | ||
|
|
@@ -757,8 +782,14 @@ class Analyzer( | |
| try { | ||
| expr transformUp { | ||
| case GetColumnByOrdinal(ordinal, _) => plan.output(ordinal) | ||
| case u @ UnresolvedAttribute(nameParts) => | ||
| withPosition(u) { plan.resolve(nameParts, resolver).getOrElse(u) } | ||
| case u @ UnresolvedAttribute(nameParts, targetPlanIdOpt) => | ||
| withPosition(u) { | ||
| targetPlanIdOpt match { | ||
| case Some(targetPlanId) => | ||
| resolveExpressionFromSpecificLogicalPlan(nameParts, plan, targetPlanId) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The resolved attribute must be among the output of children, is it guaranteed here?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thank you for taking a look. |
||
| case None => plan.resolve(nameParts, resolver).getOrElse(u) | ||
| } | ||
| } | ||
| case UnresolvedExtractValue(child, fieldName) if child.resolved => | ||
| ExtractValue(child, fieldName, resolver) | ||
| } | ||
|
|
@@ -986,12 +1017,17 @@ class Analyzer( | |
| plan transformDown { | ||
| case q: LogicalPlan if q.childrenResolved && !q.resolved => | ||
| q transformExpressions { | ||
| case u @ UnresolvedAttribute(nameParts) => | ||
| case u @ UnresolvedAttribute(nameParts, targetPlanIdOpt) => | ||
| withPosition(u) { | ||
| try { | ||
| outer.resolve(nameParts, resolver) match { | ||
| case Some(outerAttr) => OuterReference(outerAttr) | ||
| case None => u | ||
| targetPlanIdOpt match { | ||
| case Some(targetPlanId) => | ||
| resolveExpressionFromSpecificLogicalPlan(nameParts, outer, targetPlanId) | ||
| case None => | ||
| outer.resolve(nameParts, resolver) match { | ||
| case Some(outerAttr) => OuterReference(outerAttr) | ||
| case None => u | ||
| } | ||
| } | ||
| } catch { | ||
| case _: AnalysisException => u | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -627,7 +627,10 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { | |
| Row(id, name, age, salary) | ||
| }.toSeq) | ||
| assert(df.schema.map(_.name) === Seq("id", "name", "age", "salary")) | ||
| assert(df("id") == person("id")) | ||
| val dfAnalyzer = df.sparkSession.sessionState.analyzer | ||
| val personAnalyzer = person.sparkSession.sessionState.analyzer | ||
| assert(dfAnalyzer.resolveExpression(df("id").expr, df.queryExecution.analyzed) == | ||
| personAnalyzer.resolveExpression(person("id").expr, person.queryExecution.analyzed)) | ||
| } | ||
|
|
||
| test("drop top level columns that contains dot") { | ||
|
|
@@ -1601,6 +1604,28 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { | |
| assert(df.persist.take(1).apply(0).toSeq(100).asInstanceOf[Long] == 100) | ||
| } | ||
|
|
||
| test("""SPARK-17154: df("column_name") should return correct result when we do self-join""") { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What happens when: As there are two plans with the same plan id, the breadth-first search will get one plan among them. So df("col") will be resolved. However, I think in this case, we should have an ambiguous error message.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this is a good question! I'm also thinking about this. If a plan id matches more than one sub-tree in the logical plan, should we just fail the query instead of using BFS to pick the first one?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, direct-self-join (means both child Datasets are same) is still ambiguous. I'm wondering a direct-self-join like df.join(df, , ) is similar to a query like as follows. SELECT ... FROM my_table df join my_table df on ; Those queries should not be valid so I also think we shouldn't allow users to join two same Datasets and warn to duplicate the Dataset if they intend to do direct-self-join.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
If logical-plan on the right side is copied by
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Although I can't immediately think out the actual use case the self-join of two same Datasets, I am still wondering do we want to disallow it? Conceptually, it should work, even you can't select columns from it due to ambiguousness. But I think you can still save it or do other operators on it.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't think we should support a self-join of the same Dataset/DateFrame of the same name. That is,
should be blocked. We can ask the user to express it as
, which is clearer. We certainly must not support
, which blindly put |
||
| val df = Seq( | ||
| (1, "a", "A"), | ||
| (2, "b", "B"), | ||
| (3, "c", "C"), | ||
| (4, "d", "D"), | ||
| (5, "e", "E")).toDF("col1", "col2", "col3") | ||
| val filtered = df.filter("col1 != 3").select("col1", "col2") | ||
| val joined = filtered.join(df, filtered("col1") === df("col1"), "inner") | ||
| val selected1 = joined.select(df("col3")) | ||
|
|
||
| checkAnswer(selected1, Row("A") :: Row("B") :: Row("D") :: Row("E") :: Nil) | ||
|
|
||
| val rightOuterJoined = filtered.join(df, filtered("col1") === df("col1"), "right") | ||
| val selected2 = rightOuterJoined.select(df("col1")) | ||
|
|
||
| checkAnswer(selected2, Row(1) :: Row(2) :: Row(3) :: Row(4) :: Row(5) :: Nil) | ||
|
|
||
| val selected3 = rightOuterJoined.select(filtered("col1")) | ||
| checkAnswer(selected3, Row(1) :: Row(2) :: Row(null) :: Row(4) :: Row(5) :: Nil) | ||
| } | ||
|
|
||
| test("SPARK-17409: Do Not Optimize Query in CTAS (Data source tables) More Than Once") { | ||
| withTable("bar") { | ||
| withTempView("foo") { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't quite understand this. If it's a self-join, and
leftandrightis same plan(same plan id), then afterdedupRight,leftandrightis not same plan but still have same plan id right? How do we resolveUnresolvedAttributewith plan id? I think it's still ambiguous.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ah I understand now. Different dataframes must have different logical plans, the problem we are trying to fix is indirect self-join. For indirect self-join, the
leftandrighthere must be different plans and have different plan ids.