-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-27547][SQL] Fix DataFrame self-join problems #24442
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
Conversation
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.
drop should not match attribute by equality. We should use semanticEquals.
|
Test build #104832 has finished for PR 24442 at commit
|
fde7131 to
1539fa8
Compare
mgaido91
left a comment
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.
thanks for starting this and pinging me @cloud-fan. Please may you also add a Close #... for the PRs attempting to fix this? I am pretty sure there is one open by me for instance. Thanks.
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.
Here I am missing something. Sorry if the question is dumb, but I can't get why do we need to add this "special" node for it?
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.
to attach the dataset id to the logical plan, so that we can resolve column reference later.
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'd rather add the datasetId info to the LogicalPlan and avoid introducing a new plan here. I think it would be easier in this way to generalize this approach to other cases when the same problem may arise. What do you think?
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.
How can we add dataset id to every kind of LogicalPlan?
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 think that could be an option. And for the moment we could add it only to the child/children of a join since we only need it there. But I see there is no guarantee that the plan(s) are not replaces/removed during the analysis/optimization phase, so it may not be doable indeed.
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.
do we really need this column prefix? We use it only to avoid a scan, but we get it with a scan using indexOf, so I think we can get rid of it
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.
when we resolve column reference, we have no idea what's the corresponding column even if we know the corresponding dataframe. That's why we need the col position info.
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 am not a big fan of using special strings as markers, I'd rather introduce a new attribute to carry on the information.
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.
It's at plan level, we can't use attributes to carry information here.
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 meant adding a new attribute to the SubqueryPlan case class, why isn't it possible?
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.
It's possible, but I'm a little worried about changing the constructor of a widely used plan node like SubqueryAlais. It's also a little weird to put the dataset concept into the catalyst module.
Another idea I've thought about is adding a new no-op plan node to carry the dataset id. I gave it up because no-op node can be troublesome according to the past experience with ResolvedHint and AnalysisBarrier.
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.
Another idea I've thought about is adding a new no-op plan node to carry the dataset id
Yes, that's exactly the reason why I'd like to avoid adding this node. I think I'd prefer a no-op node if this is really needed actually, since this is used also in other places and we may introduce side-effects using it in a different way from what it is intended to do now.
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.
@cloud-fan I was thinking: in order to avoid adding another "placeholder" plan, since here we are dealing only with joins, what about adding a leftDatasetId and rightDatasetId to the join operator?
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.
This works, but I'd rather hack into SubqueryAlias instead of Join, as Join node is more widely used...
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.
and things can get tricky if join reorder happens.
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.
and things can get tricky if join reorder happens.
yes, I also thought of this, but still I am not very confident about using SubqueryAlias: that node is meant for a different purpose and I think ew already have things used in a hacky way and this makes more error prone any future change, because we need to think not only to the logic but also we need to remember all the "hacks" done with it. Honestly rather than using a SubqueryAlias, if no other option works, I'd prefer adding a new node (maybe a sublass of it if we need common functionalities).
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.
IMO, SubqueryAlias might be safer than new node. Because it is well known to Spark SQL community. A new node which isn't aware by others is more dangerous.
|
Test build #104835 has finished for PR 24442 at commit
|
|
Test build #104834 has finished for PR 24442 at commit
|
|
Oh, @cloud-fan . SparkR failure seems to be consistent and might be relevant. |
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.
How can a column reference point to more than one actual columns? A column reference is determined by dataset id and column position. It can't have more than one column positions or dataset ids.
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.
e.g. df1.join(df1).select(df1("id"))
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.
It might point to multiple columns, but those columns are the same column. For example, df1("id") points to columns in both left and right plans, but the pointed columns are the same. Can a column reference point to different columns? If not, this can work without ambiguousColRefs, and the code can be simplified?
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.
think about outer joins. Columns from different join sides are always different columns.
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.
@cloud-fan Could we add some tests for nested columns ? In case of nested columns, do we need to carry more metadata ?
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.
The problem only occurs for AttributeReference, which top level columns. I think nested column is totally unrelated here.
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.
@cloud-fan Ah.. ok. Thanks..
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.
Could you add SPARK-27547 at the test case name?
|
Test build #104856 has finished for PR 24442 at commit
|
|
retest this please |
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.
@cloud-fan Nit: Could this check be done by the caller once for the join or do you anticipate more callers to this function in the future other than prepareJoinPlan ?
|
sorry @cloud-fan , just to have a better understanding of your solution, may you please explain me why this is a better approach than the one in #21449? |
|
The basic idea is the same: assign a globally unique id to dataset, and carry the dataset id in the column reference(the This PR does one more thing: carry the dataset id in the logical plan of dataset in case of self-join. This makes the solution more powerful. #21449 can only resolve column reference with the current datasets, e.g. However, the hack in BTW, I think we can extend the solution in this PR with more complex ambiguity resolving logic, so that we can remove the hack in |
|
thanks @cloud-fan, I didn't think of that case! |
|
Test build #104861 has finished for PR 24442 at commit
|
|
Test build #104906 has finished for PR 24442 at commit
|
|
@cloud-fan The changes look good to me. |
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.
As indicated by the change in Analyzer, after this two attributes we considered the same, now are different if we compare it without canonicalization.
Not sure if it will be an issue. But it is counterintuitive.
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.
Internally, I think we should never compare 2 attributes by equal not semanticEqual.
Externally, if users rely on the equality of Column, yes this PR may break it under some cases. If this matters, I can add a migration guide or update Column.equal.
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 feel that metadata is not well aware. It might be confusing to some people that the comparison fails if they know it works before. An update to migration guide or Column.equal both sounds good.
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.
Can we check this between line 93 and 94? For the wrong ref, it seems that we don't need to keep it in ambiguousColRefs. The bug can hide due to filterNot(ambiguousColRefs.contains) without showing this warning.
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.
This is much better than the current status.
One side effect is that we start to distinguish the df2's rows in UNIONed dataframes like the following, too. Should we allow this behavior?
scala> val df1 = spark.range(3)
scala> val df2 = df1.filter($"id" > 0)
scala> df1.union(df2).join(df1, df1("id") > df2("id")).show
+---+---+
| id| id|
+---+---+
| 0| 1|
| 0| 2|
| 1| 2|
| 1| 2|
+---+---+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.
To be honest I don't really know what's the expected result here. df1("id") should point to the right side df1 of the join, and seems it's OK if df2(id) points to the unioned df. Before this PR this query returns no result, which is confusing as well.
If we change Union.output and generate new attributes for it, then this query throws analysis exception, which seems more reasonable.
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
Outdated
Show resolved
Hide resolved
|
Test build #105029 has finished for PR 24442 at commit
|
|
Could you fix the UT failure please? |
|
Test build #105142 has finished for PR 24442 at commit
|
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.
@cloud-fan , @gatorsmile , @rxin .
Could you split this Column change into another JIRA issue?
This looks required but orthogonal. In addition, although this PR provides a new configuration, spark.sql.analyzer.resolveDatasetColumnReference, we cannot undo this behavior change of Column class. The following is the behavior change which might affect new optimizers. I'm not saying this is good or bed. What I mean is that this need to be considered separately.
scala> spark.version
res0: String = 2.4.3
scala> rand(0).equals(rand(0))
res1: Boolean = true
scala> ($"a" + 1 + 2 + 3).equals($"a" + 3 + 2 + 1)
res2: Boolean = falsescala> spark.version // This PR
res0: String = 3.0.0-SNAPSHOT
scala> rand(0).equals(rand(0))
res1: Boolean = false
scala> ($"a" + 1 + 2 + 3).equals($"a" + 3 + 2 + 1)
res2: Boolean = trueThere 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 made a new change to avoid this issue completely.
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.
Thanks!
|
Test build #105215 has finished for PR 24442 at commit
|
|
Test build #105288 has finished for PR 24442 at commit
|
|
Test build #105437 has finished for PR 24442 at commit
|
|
Retest this please. |
|
Test build #105748 has finished for PR 24442 at commit
|
|
Retest this please. |
|
Test build #105788 has finished for PR 24442 at commit
|
|
closed in favor of #25107 |
This is an alternative solution of apache#24442 . It fails the query if ambiguous self join is detected, instead of trying to disambiguate it. The problem is that, it's hard to come up with a reasonable rule to disambiguate, the rule proposed by apache#24442 is mostly a heuristic. This is a long-standing bug and I've seen many people complaining about it in JIRA/dev list. A typical example: ``` val df1 = … val df2 = df1.filter(...) df1.join(df2, df1("a") > df2("a")) // returns empty result ``` The root cause is, `Dataset.apply` is so powerful that users think it returns a column reference which can point to the column of the Dataset at anywhere. This is not true in many cases. `Dataset.apply` returns an `AttributeReference` . Different Datasets may share the same `AttributeReference`. In the example above, `df2` adds a Filter operator above the logical plan of `df1`, and the Filter operator reserves the output `AttributeReference` of its child. This means, `df1("a")` is exactly the same as `df2("a")`, and `df1("a") > df2("a")` always evaluates to false. We can reuse the infra in apache#24442 : 1. each Dataset has a globally unique id. 2. the `AttributeReference` returned by `Dataset.apply` carries the ID and column position(e.g. 3rd column of the Dataset) via metadata. 3. the logical plan of a `Dataset` carries the ID via `TreeNodeTag` When self-join happens, the analyzer asks the right side plan of join to re-generate output attributes with new exprIds. Based on it, a simple rule to detect ambiguous self join is: 1. find all column references (i.e. `AttributeReference`s with Dataset ID and col position) in the root node of a query plan. 2. for each column reference, traverse the query plan tree, find a sub-plan that carries Dataset ID and the ID is the same as the one in the column reference. 3. get the corresponding output attribute of the sub-plan by the col position in the column reference. 4. if the corresponding output attribute has a different exprID than the column reference, then it means this sub-plan is on the right side of a self-join and has regenerated its output attributes. This is an ambiguous self join because the column reference points to a table being self-joined. existing tests and new test cases Closes apache#25107 from cloud-fan/new-self-join. Authored-by: Wenchen Fan <wenchen@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
This is an alternative solution of apache#24442 . It fails the query if ambiguous self join is detected, instead of trying to disambiguate it. The problem is that, it's hard to come up with a reasonable rule to disambiguate, the rule proposed by apache#24442 is mostly a heuristic. This is a long-standing bug and I've seen many people complaining about it in JIRA/dev list. A typical example: ``` val df1 = … val df2 = df1.filter(...) df1.join(df2, df1("a") > df2("a")) // returns empty result ``` The root cause is, `Dataset.apply` is so powerful that users think it returns a column reference which can point to the column of the Dataset at anywhere. This is not true in many cases. `Dataset.apply` returns an `AttributeReference` . Different Datasets may share the same `AttributeReference`. In the example above, `df2` adds a Filter operator above the logical plan of `df1`, and the Filter operator reserves the output `AttributeReference` of its child. This means, `df1("a")` is exactly the same as `df2("a")`, and `df1("a") > df2("a")` always evaluates to false. We can reuse the infra in apache#24442 : 1. each Dataset has a globally unique id. 2. the `AttributeReference` returned by `Dataset.apply` carries the ID and column position(e.g. 3rd column of the Dataset) via metadata. 3. the logical plan of a `Dataset` carries the ID via `TreeNodeTag` When self-join happens, the analyzer asks the right side plan of join to re-generate output attributes with new exprIds. Based on it, a simple rule to detect ambiguous self join is: 1. find all column references (i.e. `AttributeReference`s with Dataset ID and col position) in the root node of a query plan. 2. for each column reference, traverse the query plan tree, find a sub-plan that carries Dataset ID and the ID is the same as the one in the column reference. 3. get the corresponding output attribute of the sub-plan by the col position in the column reference. 4. if the corresponding output attribute has a different exprID than the column reference, then it means this sub-plan is on the right side of a self-join and has regenerated its output attributes. This is an ambiguous self join because the column reference points to a table being self-joined. existing tests and new test cases Closes apache#25107 from cloud-fan/new-self-join. Authored-by: Wenchen Fan <wenchen@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
What changes were proposed in this pull request?
This is a long-standing bug and I've seen many people complaining about it in JIRA/dev list.
A typical example:
The root cause is,
Dataset.applyis so powerful that users think it returns a column reference which can point to the column of the Dataset at anywhere. This is not true in many cases.Dataset.applyreturns anAttributeReference. Different Datasets may share the sameAttributeReference. In the example above,df2adds a Filter operator above the logical plan ofdf1, and the Filter operator reserves the outputAttributeReferenceof its child. This means,df1("a")is exactly the same asdf2("a"), anddf1("a") > df2("a")always evaluates to false.The key problem here is, when analyzer resolves self-join by assigning new expr IDs to the conflicting attributes in the right side of the join, how can we map the Dataset column reference to the new attributes of the right side?
The proposal here is:
AttributeReferencereturned byDataset.applycarries the ID and column position(e.g. 3rd column of the Dataset) via metadataDataset.join, the left and right side of the join node carry the ID viaSubqueryAlias.AttributeReferences that have dataset id, and traverses the logical plan to find the correspondingSubqueryAliaswith the same dataset id, to get the actualAttributeReference.When analyzer resolves self-join, it transforms down the right side plan to find nodes like
MultiInstanceRelationto generate attributes with new exprId. If the right side is aSubqueryAlias, its output will be the deduplicatedAttributeReferences after self-join is resolved.When we resolve dataset column reference(
AttributeReferencewith dataset id and col position), and find a matchingSubqueryAliaswith the same dataset id, we can replace the column reference with the correspondingAttributeReferenceofSubqueryAlias.How was this patch tested?
new test cases