-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-25951][SQL] Ignore aliases for distributions and orderings #22957
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
b566818
2b00f35
0491249
3831be0
6c93e70
a306465
13aef71
5c6b9fc
0aaedd8
6eee1e4
5bca5e3
1f797df
bf1d04a
e4f617f
bca3e87
0f68a41
952a2c2
9af290e
778ede3
5904cf9
205f1b7
f47d5df
09b9981
69f9d5e
75ef545
df3394c
78d92bc
47a8f71
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 |
|---|---|---|
| @@ -0,0 +1,118 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.sql.execution | ||
|
|
||
| import org.apache.spark.sql.catalyst.analysis.CleanupAliases | ||
| import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression} | ||
| import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection, UnknownPartitioning} | ||
|
|
||
| /** | ||
| * Trait for plans which can produce an output partitioned by aliased attributes of their child. | ||
| * It rewrites the partitioning attributes of the child with the corresponding new ones which are | ||
| * exposed in the output of this plan. It can avoid the presence of redundant shuffles in queries | ||
| * caused by the rename of an attribute among the partitioning ones, eg. | ||
| * | ||
| * spark.range(10).selectExpr("id AS key", "0").repartition($"key").createTempView("df1") | ||
| * spark.range(10).selectExpr("id AS key", "0").repartition($"key").createTempView("df2") | ||
| * sql("set spark.sql.autoBroadcastJoinThreshold=-1") | ||
| * sql(""" | ||
| * SELECT * FROM | ||
| * (SELECT key AS k from df1) t1 | ||
| * INNER JOIN | ||
| * (SELECT key AS k from df2) t2 | ||
| * ON t1.k = t2.k | ||
| * """).explain | ||
| * | ||
| * == Physical Plan == | ||
| * *SortMergeJoin [k#21L], [k#22L], Inner | ||
| * :- *Sort [k#21L ASC NULLS FIRST], false, 0 | ||
| * : +- Exchange hashpartitioning(k#21L, 200) // <--- Unnecessary shuffle operation | ||
| * : +- *Project [key#2L AS k#21L] | ||
| * : +- Exchange hashpartitioning(key#2L, 200) | ||
| * : +- *Project [id#0L AS key#2L] | ||
| * : +- *Range (0, 10, step=1, splits=Some(4)) | ||
| * +- *(4) Sort [k#22L ASC NULLS FIRST], false, 0 | ||
| * +- *(4) Project [key#8L AS k#22L] | ||
| * +- ReusedExchange [key#8L], Exchange hashpartitioning(key#2L, 200) | ||
| */ | ||
| trait AliasAwareOutputPartitioning extends UnaryExecNode { | ||
|
|
||
| /** | ||
| * `Seq` of `Expression`s which define the ouput of the node. | ||
| */ | ||
| protected def outputExpressions: Seq[NamedExpression] | ||
|
|
||
| /** | ||
| * Returns the valid `Partitioning`s for the node w.r.t its output and its expressions. | ||
| */ | ||
| final override def outputPartitioning: Partitioning = { | ||
| child.outputPartitioning match { | ||
| case partitioning: Expression => | ||
| // Creates a sequence of tuples where the first element is an `Attribute` referenced in the | ||
| // partitioning expression of the child and the second is a sequence of all its aliased | ||
| // occurrences in the node output. If there is no occurrence of an attribute in the output, | ||
| // the second element of the tuple for it will be an empty `Seq`. If the attribute, | ||
| // instead, is only present as is in the output, there will be no entry for it. | ||
| // Eg. if the partitioning is RangePartitioning('a) and the node output is "a, 'a as a1, | ||
| // a' as a2", then exprToEquiv will contain the tuple ('a, Seq('a, 'a as a1, 'a as a2)). | ||
| val exprToEquiv = partitioning.references.map { attr => | ||
|
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. can you explain what's going on here? The code is a little hard to follow.
Contributor
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. sure, let me add some comments. Thanks. |
||
| attr -> outputExpressions.filter(e => | ||
| CleanupAliases.trimAliases(e).semanticEquals(attr)) | ||
| }.filterNot { case (attr, exprs) => | ||
| exprs.size == 1 && exprs.forall(_ == attr) | ||
| } | ||
| val initValue = partitioning match { | ||
| case PartitioningCollection(partitionings) => partitionings | ||
| case other => Seq(other) | ||
| } | ||
| // Replace all the aliased expressions detected earlier with all their corresponding | ||
| // occurrences. This may produce many valid partitioning expressions from a single one. | ||
| // Eg. in the example above, this would produce a `Seq` of 3 `RangePartitioning`, namely: | ||
| // `RangePartitioning('a)`, `RangePartitioning('a1)`, `RangePartitioning('a2)`. | ||
| val validPartitionings = exprToEquiv.foldLeft(initValue) { | ||
| case (partitionings, (toReplace, equivalents)) => | ||
| if (equivalents.isEmpty) { | ||
| // Remove from the partitioning expression the attribute which is not present in the | ||
| // node output | ||
| partitionings.map(_.pruneInvalidAttribute(toReplace)) | ||
| } else { | ||
| partitionings.flatMap { | ||
| case p: Expression if p.references.contains(toReplace) => | ||
| equivalents.map { equiv => | ||
| p.transformDown { | ||
| case e if e == toReplace => equiv.toAttribute | ||
| }.asInstanceOf[Partitioning] | ||
| } | ||
| case other => Seq(other) | ||
| } | ||
| } | ||
| }.distinct | ||
| if (validPartitionings.size == 1) { | ||
| validPartitionings.head | ||
| } else { | ||
| validPartitionings.filterNot(_.isInstanceOf[UnknownPartitioning]) match { | ||
| case Seq() => PartitioningCollection(validPartitionings) | ||
| case Seq(knownPartitioning) => knownPartitioning | ||
| case knownPartitionings => PartitioningCollection(knownPartitionings) | ||
| } | ||
|
|
||
| } | ||
| case other => other | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -25,7 +25,9 @@ import org.scalatest.exceptions.TestFailedException | |
| import org.apache.spark.{SparkException, TaskContext} | ||
| import org.apache.spark.sql.catalyst.ScroogeLikeExample | ||
| import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder} | ||
| import org.apache.spark.sql.catalyst.expressions.AttributeReference | ||
| import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} | ||
| import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning | ||
| import org.apache.spark.sql.catalyst.util.sideBySide | ||
| import org.apache.spark.sql.execution.{LogicalRDD, RDDScanExec, SQLExecution} | ||
| import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} | ||
|
|
@@ -1595,6 +1597,38 @@ class DatasetSuite extends QueryTest with SharedSQLContext { | |
| Seq(Row("Amsterdam"))) | ||
| } | ||
|
|
||
| test("SPARK-25951: avoid redundant shuffle on rename") { | ||
|
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'm a little worried about introducing such a big change for such a corner case. Looking at this test case, it seems updating the cc @maryannxue
Contributor
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.
Basically, that is what is done. The diff is quite big, that's true, but mostly are tests. Is there something specific you are worried about?
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'm worried about
Contributor
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. it is still useful because it may happen a used writes something like: (please notice that despite it makes no sense written in SQL, maybe with other APIs is easier to happen). And especially it is useful when updating the |
||
| withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { | ||
| val N = 10 | ||
| val t1 = spark.range(N).selectExpr("floor(id/4) as k1") | ||
| val t2 = spark.range(N).selectExpr("floor(id/4) as k2") | ||
|
|
||
| val agg1 = t1.groupBy("k1").agg(count(lit("1")).as("cnt1")) | ||
| val agg2 = t2.groupBy("k2").agg(count(lit("1")).as("cnt2")).withColumnRenamed("k2", "k3") | ||
| val finalPlan = agg1.join(agg2, $"k1" === $"k3") | ||
| val exchanges = finalPlan.queryExecution.executedPlan.collect { | ||
| case se: ShuffleExchangeExec => se | ||
| } | ||
| assert(exchanges.size == 2) | ||
| assert(!exchanges.exists(_.newPartitioning match { | ||
| case HashPartitioning(Seq(a: AttributeReference), _) => a.name == "k3" | ||
| case _ => false | ||
| })) | ||
|
|
||
| // In this case the requirement is not satisfied | ||
| val agg3 = t2.groupBy("k2").agg(count(lit("1")).as("cnt2")).withColumn("k3", $"k2" + 1) | ||
| val finalPlan2 = agg1.join(agg3, $"k1" === $"k3") | ||
| val exchanges2 = finalPlan2.queryExecution.executedPlan.collect { | ||
| case se: ShuffleExchangeExec => se | ||
| } | ||
| assert(exchanges2.size == 3) | ||
| assert(exchanges2.exists(_.newPartitioning match { | ||
| case HashPartitioning(Seq(a: AttributeReference), _) => a.name == "k3" | ||
| case _ => false | ||
| })) | ||
| } | ||
| } | ||
|
|
||
| test("SPARK-24762: Enable top-level Option of Product encoders") { | ||
| val data = Seq(Some((1, "a")), Some((2, "b")), None) | ||
| val ds = data.toDS() | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
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.
Let's add comments to explain it.
HashPartitioning('a, 'b)with output expressions'a as 'a1, should produceUnknownPartitioninginstead ofHashPartitioning('a1), which is wrong.