-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL #24706
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 23 commits
52f0222
1c665d2
a9b4209
cbfbc4e
4255421
a656e42
ac0794d
62af5be
77a668b
3e85e74
bd6a364
9eaf307
e2fa8e3
55450e9
4b0755d
6e547d7
baef964
ec59f88
9af4eb1
5b5ac2e
da33bd7
5688cb4
a40b771
37905f5
eb8fe75
4481085
8570ec0
237c067
d6e040b
e265104
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 |
|---|---|---|
|
|
@@ -317,76 +317,92 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { | |
| */ | ||
| def mapChildren(f: BaseType => BaseType): BaseType = { | ||
| if (children.nonEmpty) { | ||
| var changed = false | ||
| def mapChild(child: Any): Any = child match { | ||
| case arg: TreeNode[_] if containsChild(arg) => | ||
| val newChild = f(arg.asInstanceOf[BaseType]) | ||
| if (!(newChild fastEquals arg)) { | ||
| changed = true | ||
| newChild | ||
| } else { | ||
| arg | ||
| } | ||
| case tuple@(arg1: TreeNode[_], arg2: TreeNode[_]) => | ||
| val newChild1 = if (containsChild(arg1)) { | ||
| f(arg1.asInstanceOf[BaseType]) | ||
| } else { | ||
| arg1.asInstanceOf[BaseType] | ||
| } | ||
| mapProductElements(f, applyToAll = false) | ||
| } else { | ||
| this | ||
| } | ||
| } | ||
|
|
||
| val newChild2 = if (containsChild(arg2)) { | ||
| f(arg2.asInstanceOf[BaseType]) | ||
| } else { | ||
| arg2.asInstanceOf[BaseType] | ||
| } | ||
| /** | ||
| * Returns a copy of this node where `f` has been applied to all applicable `TreeNode` elements | ||
| * in the productIterator. | ||
| * @param f the transform function to be applied on applicable `TreeNode` elements. | ||
| * @param applyToAll If true, the transform function will be applied to all `TreeNode` elements | ||
| * even for non-child elements; otherwise, the function will only be applied | ||
| * on children nodes. Also, when this is true, a copy of this node will be | ||
| * returned even if no elements have been changed. | ||
| */ | ||
| private def mapProductElements( | ||
| f: BaseType => BaseType, | ||
| applyToAll: Boolean): BaseType = { | ||
| var changed = false | ||
|
|
||
| if (!(newChild1 fastEquals arg1) || !(newChild2 fastEquals arg2)) { | ||
| changed = true | ||
| (newChild1, newChild2) | ||
| } else { | ||
| tuple | ||
| } | ||
| case other => other | ||
| } | ||
| def mapChild(child: Any): Any = child match { | ||
|
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. mapElement? |
||
| case arg: TreeNode[_] if applyToAll || containsChild(arg) => | ||
|
||
| val newChild = f(arg.asInstanceOf[BaseType]) | ||
| if (applyToAll || !(newChild fastEquals arg)) { | ||
| changed = true | ||
| newChild | ||
| } else { | ||
| arg | ||
| } | ||
| case tuple @ (arg1: TreeNode[_], arg2: TreeNode[_]) => | ||
| val newChild1 = if (applyToAll || containsChild(arg1)) { | ||
| f(arg1.asInstanceOf[BaseType]) | ||
| } else { | ||
| arg1.asInstanceOf[BaseType] | ||
| } | ||
|
|
||
| val newChild2 = if (applyToAll || containsChild(arg2)) { | ||
| f(arg2.asInstanceOf[BaseType]) | ||
| } else { | ||
| arg2.asInstanceOf[BaseType] | ||
| } | ||
|
|
||
| if (applyToAll || !(newChild1 fastEquals arg1) || !(newChild2 fastEquals arg2)) { | ||
| changed = true | ||
| (newChild1, newChild2) | ||
| } else { | ||
| tuple | ||
| } | ||
| case other => other | ||
| } | ||
|
|
||
| val newArgs = mapProductIterator { | ||
| case arg: TreeNode[_] if containsChild(arg) => | ||
| val newArgs = mapProductIterator { | ||
| case arg: TreeNode[_] if applyToAll || containsChild(arg) => | ||
| val newChild = f(arg.asInstanceOf[BaseType]) | ||
| if (applyToAll || !(newChild fastEquals arg)) { | ||
| changed = true | ||
| newChild | ||
| } else { | ||
| arg | ||
| } | ||
| case Some(arg: TreeNode[_]) if applyToAll || containsChild(arg) => | ||
| val newChild = f(arg.asInstanceOf[BaseType]) | ||
| if (applyToAll || !(newChild fastEquals arg)) { | ||
| changed = true | ||
| Some(newChild) | ||
| } else { | ||
| Some(arg) | ||
| } | ||
| case m: Map[_, _] => m.mapValues { | ||
| case arg: TreeNode[_] if applyToAll || containsChild(arg) => | ||
| val newChild = f(arg.asInstanceOf[BaseType]) | ||
| if (!(newChild fastEquals arg)) { | ||
| if (applyToAll || !(newChild fastEquals arg)) { | ||
| changed = true | ||
| newChild | ||
| } else { | ||
| arg | ||
| } | ||
| case Some(arg: TreeNode[_]) if containsChild(arg) => | ||
| val newChild = f(arg.asInstanceOf[BaseType]) | ||
| if (!(newChild fastEquals arg)) { | ||
| changed = true | ||
| Some(newChild) | ||
| } else { | ||
| Some(arg) | ||
| } | ||
| case m: Map[_, _] => m.mapValues { | ||
| case arg: TreeNode[_] if containsChild(arg) => | ||
| val newChild = f(arg.asInstanceOf[BaseType]) | ||
| if (!(newChild fastEquals arg)) { | ||
| changed = true | ||
| newChild | ||
| } else { | ||
| arg | ||
| } | ||
| case other => other | ||
| }.view.force // `mapValues` is lazy and we need to force it to materialize | ||
| case d: DataType => d // Avoid unpacking Structs | ||
| case args: Stream[_] => args.map(mapChild).force // Force materialization on stream | ||
| case args: Iterable[_] => args.map(mapChild) | ||
| case nonChild: AnyRef => nonChild | ||
| case null => null | ||
| } | ||
| if (changed) makeCopy(newArgs) else this | ||
| } else { | ||
| this | ||
| case other => other | ||
| }.view.force // `mapValues` is lazy and we need to force it to materialize | ||
| case d: DataType => d // Avoid unpacking Structs | ||
| case args: Stream[_] => args.map(mapChild).force // Force materialization on stream | ||
| case args: Iterable[_] => args.map(mapChild) | ||
| case nonChild: AnyRef => nonChild | ||
| case null => null | ||
| } | ||
| if (applyToAll || changed) makeCopy(newArgs, applyToAll) else this | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -402,9 +418,20 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { | |
| * that are not present in the productIterator. | ||
| * @param newArgs the new product arguments. | ||
| */ | ||
| def makeCopy(newArgs: Array[AnyRef]): BaseType = attachTree(this, "makeCopy") { | ||
| def makeCopy(newArgs: Array[AnyRef]): BaseType = makeCopy(newArgs, allowEmptyArgs = false) | ||
|
|
||
| /** | ||
| * Creates a copy of this type of tree node after a transformation. | ||
| * Must be overridden by child classes that have constructor arguments | ||
| * that are not present in the productIterator. | ||
| * @param newArgs the new product arguments. | ||
| * @param allowEmptyArgs whether to allow argument list to be empty. | ||
| */ | ||
| private def makeCopy( | ||
| newArgs: Array[AnyRef], | ||
| allowEmptyArgs: Boolean): BaseType = attachTree(this, "makeCopy") { | ||
| // Skip no-arg constructors that are just there for kryo. | ||
| val ctors = getClass.getConstructors.filter(_.getParameterTypes.size != 0) | ||
| val ctors = getClass.getConstructors.filter(allowEmptyArgs || _.getParameterTypes.size != 0) | ||
| if (ctors.isEmpty) { | ||
| sys.error(s"No valid constructor for $nodeName") | ||
| } | ||
|
|
@@ -447,6 +474,10 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { | |
| } | ||
| } | ||
|
|
||
| override def clone(): BaseType = { | ||
| mapProductElements(_.clone(), applyToAll = true) | ||
| } | ||
|
|
||
| /** | ||
| * Returns the name of this type of TreeNode. Defaults to the class name. | ||
| * Note that we remove the "Exec" suffix for physical operators here. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -292,6 +292,12 @@ object SQLConf { | |
| .bytesConf(ByteUnit.BYTE) | ||
| .createWithDefault(64 * 1024 * 1024) | ||
|
|
||
| val RUNTIME_REOPTIMIZATION_ENABLED = | ||
| buildConf("spark.sql.runtime.reoptimization.enabled") | ||
| .doc("When true, enable runtime query re-optimization.") | ||
| .booleanConf | ||
| .createWithDefault(false) | ||
|
|
||
| val ADAPTIVE_EXECUTION_ENABLED = buildConf("spark.sql.adaptive.enabled") | ||
| .doc("When true, enable adaptive query execution.") | ||
| .booleanConf | ||
|
||
|
|
@@ -1882,7 +1888,10 @@ class SQLConf extends Serializable with Logging { | |
| def targetPostShuffleInputSize: Long = | ||
| getConf(SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE) | ||
|
|
||
| def adaptiveExecutionEnabled: Boolean = getConf(ADAPTIVE_EXECUTION_ENABLED) | ||
| def runtimeReoptimizationEnabled: Boolean = getConf(RUNTIME_REOPTIMIZATION_ENABLED) | ||
|
|
||
| def adaptiveExecutionEnabled: Boolean = | ||
| getConf(ADAPTIVE_EXECUTION_ENABLED) && !getConf(RUNTIME_REOPTIMIZATION_ENABLED) | ||
|
|
||
| def minNumPostShufflePartitions: Int = | ||
| getConf(SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -40,9 +40,11 @@ import org.apache.spark.sql.execution.metric.SQLMetric | |
| import org.apache.spark.sql.types.DataType | ||
|
|
||
| object SparkPlan { | ||
| // a TreeNode tag in SparkPlan, to carry its original logical plan. The planner will add this tag | ||
| // when converting a logical plan to a physical plan. | ||
| /** The original [[LogicalPlan]] from which this [[SparkPlan]] is converted. */ | ||
| val LOGICAL_PLAN_TAG = TreeNodeTag[LogicalPlan]("logical_plan") | ||
|
|
||
| /** The [[LogicalPlan]] inherited from its ancestor. */ | ||
| val LOGICAL_PLAN_INHERITED_TAG = TreeNodeTag[LogicalPlan]("logical_plan_inherited") | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -79,6 +81,35 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ | |
| super.makeCopy(newArgs) | ||
| } | ||
|
|
||
| /** | ||
| * @return The logical plan this plan is linked to. | ||
| */ | ||
| def logicalLink: Option[LogicalPlan] = | ||
| getTagValue(SparkPlan.LOGICAL_PLAN_TAG) | ||
| .orElse(getTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG)) | ||
|
|
||
| /** | ||
| * Set logical plan link recursively if unset. | ||
| */ | ||
| def setLogicalLink(logicalPlan: LogicalPlan): Unit = { | ||
| setLogicalLink(logicalPlan, false) | ||
| } | ||
|
|
||
| private def setLogicalLink(logicalPlan: LogicalPlan, inherited: Boolean = false): Unit = { | ||
| // Stop at a descendant which is the root of a sub-tree transformed from another logical node. | ||
| if (inherited && getTagValue(SparkPlan.LOGICAL_PLAN_TAG).isDefined) { | ||
| return | ||
| } | ||
|
|
||
| val tag = if (inherited) { | ||
| SparkPlan.LOGICAL_PLAN_INHERITED_TAG | ||
| } else { | ||
| SparkPlan.LOGICAL_PLAN_TAG | ||
| } | ||
| setTagValue(tag, logicalPlan) | ||
| children.foreach(_.setLogicalLink(logicalPlan, true)) | ||
|
||
| } | ||
|
|
||
| /** | ||
| * @return All metrics containing metrics of this SparkPlan. | ||
| */ | ||
|
|
||
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.
If we can then we should spin this off in a separate PR.