-
Notifications
You must be signed in to change notification settings - Fork 447
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
[CORE] Add GlutenImplicits to get FallbackSummary easily #3599
Merged
Merged
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
212 changes: 212 additions & 0 deletions
212
gluten-core/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,212 @@ | ||
/* | ||
* 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 io.glutenproject.execution.WholeStageTransformer | ||
import io.glutenproject.extension.{GlutenPlan, InMemoryTableScanHelper} | ||
|
||
import org.apache.spark.sql.{AnalysisException, Dataset} | ||
import org.apache.spark.sql.catalyst.plans.QueryPlan | ||
import org.apache.spark.sql.catalyst.plans.logical.{CommandResult, LogicalPlan} | ||
import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat | ||
import org.apache.spark.sql.execution.GlutenExplainUtils._ | ||
import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AQEShuffleReadExec, QueryStageExec} | ||
import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec | ||
import org.apache.spark.sql.execution.command.{DataWritingCommandExec, ExecutedCommandExec} | ||
import org.apache.spark.sql.execution.datasources.v2.V2CommandExec | ||
import org.apache.spark.sql.internal.SQLConf | ||
|
||
import scala.collection.mutable | ||
import scala.collection.mutable.ArrayBuffer | ||
|
||
// spotless:off | ||
/** | ||
* A helper class to get the Gluten fallback summary from a Spark [[Dataset]]. | ||
* | ||
* Note that, if AQE is enabled, but the query is not materialized, then this method will re-plan | ||
* the query execution with disabled AQE. It is a workaround to get the final plan, and it may | ||
* cause the inconsistent results with a materialized query. However, we have no choice. | ||
* | ||
* For example: | ||
* | ||
* {{{ | ||
* import org.apache.spark.sql.execution.GlutenImplicits._ | ||
* val df = spark.sql("SELECT * FROM t") | ||
* df.fallbackSummary | ||
* }}} | ||
*/ | ||
// spotless:on | ||
object GlutenImplicits { | ||
|
||
case class FallbackSummary( | ||
numGlutenNodes: Int, | ||
numFallbackNodes: Int, | ||
physicalPlanDescription: Seq[String], | ||
fallbackNodeToReason: Seq[Map[String, String]]) {} | ||
|
||
private[sql] def withSQLConf[T](pairs: (String, String)*)(f: => T): T = { | ||
val conf = SQLConf.get | ||
val (keys, values) = pairs.unzip | ||
val currentValues = keys.map { | ||
key => | ||
if (conf.contains(key)) { | ||
Some(conf.getConfString(key)) | ||
} else { | ||
None | ||
} | ||
} | ||
keys.zip(values).foreach { | ||
case (k, v) => | ||
if (SQLConf.isStaticConfigKey(k)) { | ||
throw new AnalysisException(s"Cannot modify the value of a static config: $k") | ||
} | ||
conf.setConfString(k, v) | ||
} | ||
try f | ||
finally { | ||
keys.zip(currentValues).foreach { | ||
case (key, Some(value)) => conf.setConfString(key, value) | ||
case (key, None) => conf.unsetConf(key) | ||
} | ||
} | ||
} | ||
|
||
implicit class DatasetTransformer[T](dateset: Dataset[T]) { | ||
private def isFinalAdaptivePlan(p: AdaptiveSparkPlanExec): Boolean = { | ||
val args = p.argString(Int.MaxValue) | ||
val index = args.indexOf("isFinalPlan=") | ||
assert(index >= 0) | ||
args.substring(index + "isFinalPlan=".length).trim.toBoolean | ||
} | ||
|
||
private def collectFallbackNodes(plan: QueryPlan[_]): FallbackInfo = { | ||
var numGlutenNodes = 0 | ||
val fallbackNodeToReason = new mutable.HashMap[String, String] | ||
|
||
def collect(tmp: QueryPlan[_]): Unit = { | ||
tmp.foreachUp { | ||
case _: ExecutedCommandExec => | ||
case _: CommandResultExec => | ||
case _: V2CommandExec => | ||
case _: DataWritingCommandExec => | ||
case _: WholeStageCodegenExec => | ||
case _: WholeStageTransformer => | ||
case _: InputAdapter => | ||
case _: ColumnarToRowTransition => | ||
case _: RowToColumnarTransition => | ||
case p: AdaptiveSparkPlanExec if isFinalAdaptivePlan(p) => | ||
collect(p.executedPlan) | ||
case p: AdaptiveSparkPlanExec => | ||
// if we are here that means we are inside table cache. | ||
val (innerNumGlutenNodes, innerFallbackNodeToReason) = | ||
withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { | ||
// re-plan manually to skip cached data | ||
val newSparkPlan = QueryExecution.createSparkPlan( | ||
dateset.sparkSession, | ||
dateset.sparkSession.sessionState.planner, | ||
p.inputPlan.logicalLink.get) | ||
val newExecutedPlan = QueryExecution.prepareExecutedPlan( | ||
dateset.sparkSession, | ||
newSparkPlan | ||
) | ||
processPlan( | ||
newExecutedPlan, | ||
new PlanStringConcat().append, | ||
Some(plan => collectFallbackNodes(plan))) | ||
} | ||
numGlutenNodes += innerNumGlutenNodes | ||
fallbackNodeToReason.++=(innerFallbackNodeToReason) | ||
case p: QueryStageExec => collect(p.plan) | ||
case p: GlutenPlan => | ||
numGlutenNodes += 1 | ||
p.innerChildren.foreach(collect) | ||
case i: InMemoryTableScanExec => | ||
if (InMemoryTableScanHelper.isGlutenTableCache(i)) { | ||
numGlutenNodes += 1 | ||
} else { | ||
addFallbackNodeWithReason(i, "Columnar table cache is disabled", fallbackNodeToReason) | ||
} | ||
collect(i.relation.cachedPlan) | ||
case _: AQEShuffleReadExec => // Ignore | ||
case p: SparkPlan => | ||
handleVanillaSparkPlan(p, fallbackNodeToReason) | ||
p.innerChildren.foreach(collect) | ||
case _ => | ||
} | ||
} | ||
|
||
collect(plan) | ||
(numGlutenNodes, fallbackNodeToReason.toMap) | ||
} | ||
|
||
private def collectQueryExecutionFallbackSummary(qe: QueryExecution): FallbackSummary = { | ||
var totalNumGlutenNodes = 0 | ||
var totalNumFallbackNodes = 0 | ||
val totalPhysicalPlanDescription = new ArrayBuffer[String]() | ||
val totalFallbackNodeToReason = new ArrayBuffer[Map[String, String]]() | ||
|
||
def handlePlanWithAQEAndTableCache( | ||
plan: SparkPlan, | ||
logicalPlan: LogicalPlan, | ||
isMaterialized: Boolean): Unit = { | ||
val concat = new PlanStringConcat() | ||
val collectFallbackFunc = Some(plan => collectFallbackNodes(plan)) | ||
val (numGlutenNodes, fallbackNodeToReason) = if (!isMaterialized) { | ||
withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { | ||
// AQE is not materialized, so the columnar rules are not applied. | ||
// For this case, We apply columnar rules manually with disable AQE. | ||
val qe = dateset.sparkSession.sessionState.executePlan(logicalPlan) | ||
processPlan(qe.executedPlan, concat.append, collectFallbackFunc) | ||
} | ||
} else { | ||
processPlan(plan, concat.append, collectFallbackFunc) | ||
} | ||
totalNumGlutenNodes += numGlutenNodes | ||
totalNumFallbackNodes += fallbackNodeToReason.size | ||
totalPhysicalPlanDescription.append(concat.toString()) | ||
totalFallbackNodeToReason.append(fallbackNodeToReason) | ||
} | ||
|
||
// For command-like query, e.g., `INSERT INTO TABLE ...` | ||
qe.commandExecuted.foreach { | ||
case r: CommandResult => | ||
handlePlanWithAQEAndTableCache(r.commandPhysicalPlan, r.commandLogicalPlan, true) | ||
case _ => // ignore | ||
} | ||
|
||
// For query, e.g., `SELECT * FROM ...` | ||
if (qe.executedPlan.find(_.isInstanceOf[CommandResultExec]).isEmpty) { | ||
val isMaterialized = qe.executedPlan.find { | ||
case a: AdaptiveSparkPlanExec if isFinalAdaptivePlan(a) => true | ||
case _ => false | ||
}.isDefined | ||
handlePlanWithAQEAndTableCache(qe.executedPlan, qe.analyzed, isMaterialized) | ||
} | ||
|
||
FallbackSummary( | ||
totalNumGlutenNodes, | ||
totalNumFallbackNodes, | ||
totalPhysicalPlanDescription, | ||
totalFallbackNodeToReason | ||
) | ||
} | ||
|
||
def fallbackSummary(): FallbackSummary = { | ||
collectQueryExecutionFallbackSummary(dateset.queryExecution) | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back 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.
This feature is not workable for CH backend? cc @zzcclp.
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 should work with CH backend. There are some different behavior, so I disable these tests for CH backend. e.g., velox backend would add one more project before shuffle, velox backend supports columnar cache, etc..