-
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 all 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 |
|---|---|---|
|
|
@@ -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. | ||
| */ | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution | |
|
|
||
| import java.io.Writer | ||
| import java.util.Locale | ||
| import java.util.concurrent.atomic.AtomicInteger | ||
|
|
||
| import scala.collection.mutable | ||
| import scala.util.control.NonFatal | ||
|
|
@@ -551,56 +552,6 @@ object WholeStageCodegenExec { | |
| } | ||
| } | ||
|
|
||
| object WholeStageCodegenId { | ||
| // codegenStageId: ID for codegen stages within a query plan. | ||
| // It does not affect equality, nor does it participate in destructuring pattern matching | ||
| // of WholeStageCodegenExec. | ||
| // | ||
| // This ID is used to help differentiate between codegen stages. It is included as a part | ||
| // of the explain output for physical plans, e.g. | ||
| // | ||
| // == Physical Plan == | ||
| // *(5) SortMergeJoin [x#3L], [y#9L], Inner | ||
| // :- *(2) Sort [x#3L ASC NULLS FIRST], false, 0 | ||
| // : +- Exchange hashpartitioning(x#3L, 200) | ||
| // : +- *(1) Project [(id#0L % 2) AS x#3L] | ||
| // : +- *(1) Filter isnotnull((id#0L % 2)) | ||
| // : +- *(1) Range (0, 5, step=1, splits=8) | ||
| // +- *(4) Sort [y#9L ASC NULLS FIRST], false, 0 | ||
| // +- Exchange hashpartitioning(y#9L, 200) | ||
| // +- *(3) Project [(id#6L % 2) AS y#9L] | ||
| // +- *(3) Filter isnotnull((id#6L % 2)) | ||
| // +- *(3) Range (0, 5, step=1, splits=8) | ||
| // | ||
| // where the ID makes it obvious that not all adjacent codegen'd plan operators are of the | ||
| // same codegen stage. | ||
| // | ||
| // The codegen stage ID is also optionally included in the name of the generated classes as | ||
| // a suffix, so that it's easier to associate a generated class back to the physical operator. | ||
| // This is controlled by SQLConf: spark.sql.codegen.useIdInClassName | ||
| // | ||
| // The ID is also included in various log messages. | ||
| // | ||
| // Within a query, a codegen stage in a plan starts counting from 1, in "insertion order". | ||
| // WholeStageCodegenExec operators are inserted into a plan in depth-first post-order. | ||
| // See CollapseCodegenStages.insertWholeStageCodegen for the definition of insertion order. | ||
| // | ||
| // 0 is reserved as a special ID value to indicate a temporary WholeStageCodegenExec object | ||
| // is created, e.g. for special fallback handling when an existing WholeStageCodegenExec | ||
| // failed to generate/compile code. | ||
|
|
||
| private val codegenStageCounter: ThreadLocal[Integer] = ThreadLocal.withInitial(() => 1) | ||
|
|
||
| def resetPerQuery(): Unit = codegenStageCounter.set(1) | ||
|
|
||
| def getNextStageId(): Int = { | ||
| val counter = codegenStageCounter | ||
| val id = counter.get() | ||
| counter.set(id + 1) | ||
| id | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * WholeStageCodegen compiles a subtree of plans that support codegen together into single Java | ||
| * function. | ||
|
|
@@ -824,8 +775,48 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) | |
|
|
||
| /** | ||
| * Find the chained plans that support codegen, collapse them together as WholeStageCodegen. | ||
| * | ||
| * The `codegenStageCounter` generates ID for codegen stages within a query plan. | ||
| * It does not affect equality, nor does it participate in destructuring pattern matching | ||
| * of WholeStageCodegenExec. | ||
| * | ||
| * This ID is used to help differentiate between codegen stages. It is included as a part | ||
| * of the explain output for physical plans, e.g. | ||
| * | ||
| * == Physical Plan == | ||
| * *(5) SortMergeJoin [x#3L], [y#9L], Inner | ||
| * :- *(2) Sort [x#3L ASC NULLS FIRST], false, 0 | ||
| * : +- Exchange hashpartitioning(x#3L, 200) | ||
| * : +- *(1) Project [(id#0L % 2) AS x#3L] | ||
| * : +- *(1) Filter isnotnull((id#0L % 2)) | ||
| * : +- *(1) Range (0, 5, step=1, splits=8) | ||
| * +- *(4) Sort [y#9L ASC NULLS FIRST], false, 0 | ||
| * +- Exchange hashpartitioning(y#9L, 200) | ||
| * +- *(3) Project [(id#6L % 2) AS y#9L] | ||
| * +- *(3) Filter isnotnull((id#6L % 2)) | ||
| * +- *(3) Range (0, 5, step=1, splits=8) | ||
| * | ||
| * where the ID makes it obvious that not all adjacent codegen'd plan operators are of the | ||
| * same codegen stage. | ||
| * | ||
| * The codegen stage ID is also optionally included in the name of the generated classes as | ||
| * a suffix, so that it's easier to associate a generated class back to the physical operator. | ||
| * This is controlled by SQLConf: spark.sql.codegen.useIdInClassName | ||
| * | ||
| * The ID is also included in various log messages. | ||
| * | ||
| * Within a query, a codegen stage in a plan starts counting from 1, in "insertion order". | ||
| * WholeStageCodegenExec operators are inserted into a plan in depth-first post-order. | ||
| * See CollapseCodegenStages.insertWholeStageCodegen for the definition of insertion order. | ||
| * | ||
| * 0 is reserved as a special ID value to indicate a temporary WholeStageCodegenExec object | ||
| * is created, e.g. for special fallback handling when an existing WholeStageCodegenExec | ||
| * failed to generate/compile code. | ||
| */ | ||
| case class CollapseCodegenStages(conf: SQLConf) extends Rule[SparkPlan] { | ||
| case class CollapseCodegenStages( | ||
| conf: SQLConf, | ||
| codegenStageCounter: AtomicInteger = new AtomicInteger(0)) | ||
|
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. where do we not use the default value?
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. Yeah, we could have made that part of the class body. OTOH this also works :)... |
||
| extends Rule[SparkPlan] { | ||
|
|
||
| private def supportCodegen(e: Expression): Boolean = e match { | ||
| case e: LeafExpression => true | ||
|
|
@@ -869,14 +860,13 @@ case class CollapseCodegenStages(conf: SQLConf) extends Rule[SparkPlan] { | |
| case plan if plan.output.length == 1 && plan.output.head.dataType.isInstanceOf[ObjectType] => | ||
| plan.withNewChildren(plan.children.map(insertWholeStageCodegen)) | ||
| case plan: CodegenSupport if supportCodegen(plan) => | ||
| WholeStageCodegenExec(insertInputAdapter(plan))(WholeStageCodegenId.getNextStageId()) | ||
| WholeStageCodegenExec(insertInputAdapter(plan))(codegenStageCounter.incrementAndGet()) | ||
| case other => | ||
| other.withNewChildren(other.children.map(insertWholeStageCodegen)) | ||
| } | ||
|
|
||
| def apply(plan: SparkPlan): SparkPlan = { | ||
| if (conf.wholeStageEnabled) { | ||
| WholeStageCodegenId.resetPerQuery() | ||
| insertWholeStageCodegen(plan) | ||
| } else { | ||
| plan | ||
|
|
||
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.
We probably need to update the doc for this config. It isn't enabled when runtime query re-optimization is true.
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 leave it now and see if we should use the existing config
spark.sql.adaptive.enabledinstead.