forked from NVIDIA/spark-rapids-tools
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Remove ReusedSubquery from SparkPlanGraph construction
Signed-off-by: Ahmed Hussein (amahussein) <[email protected]> Fixes NVIDIA#718 - Reused subqueries should be excluded from the metrics. - Added a common graph builder to be used for both CPU/GPU logs - Added the unit test for GPU eventlogs
- Loading branch information
1 parent
a154c0b
commit bee74bb
Showing
9 changed files
with
150 additions
and
14 deletions.
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
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
125 changes: 125 additions & 0 deletions
125
core/src/main/scala/org/apache/spark/sql/rapids/tool/util/ToolsPlanGraph.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,125 @@ | ||
/* | ||
* Copyright (c) 2024, NVIDIA CORPORATION. | ||
* | ||
* Licensed 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.rapids.tool.util | ||
|
||
import java.util.concurrent.atomic.AtomicLong | ||
|
||
import scala.collection.mutable | ||
|
||
import org.apache.spark.sql.execution.SparkPlanInfo | ||
import org.apache.spark.sql.execution.ui.{SparkPlanGraph, SparkPlanGraphCluster, SparkPlanGraphEdge, SparkPlanGraphNode, SQLPlanMetric} | ||
|
||
object ToolsPlanGraph { | ||
/** | ||
* Build a SparkPlanGraph from the root of a SparkPlan tree. | ||
*/ | ||
def apply(planInfo: SparkPlanInfo): SparkPlanGraph = { | ||
val nodeIdGenerator = new AtomicLong(0) | ||
val nodes = mutable.ArrayBuffer[SparkPlanGraphNode]() | ||
val edges = mutable.ArrayBuffer[SparkPlanGraphEdge]() | ||
val exchanges = mutable.HashMap[SparkPlanInfo, SparkPlanGraphNode]() | ||
buildSparkPlanGraphNode(planInfo, nodeIdGenerator, nodes, edges, null, null, exchanges) | ||
new SparkPlanGraph(nodes, edges) | ||
} | ||
|
||
private def processPlanInfo(nodeName: String): String = { | ||
if (nodeName.startsWith("Gpu")) { | ||
nodeName.replaceFirst("Gpu", "") | ||
} else { | ||
nodeName | ||
} | ||
} | ||
|
||
private def buildSparkPlanGraphNode( | ||
planInfo: SparkPlanInfo, | ||
nodeIdGenerator: AtomicLong, | ||
nodes: mutable.ArrayBuffer[SparkPlanGraphNode], | ||
edges: mutable.ArrayBuffer[SparkPlanGraphEdge], | ||
parent: SparkPlanGraphNode, | ||
subgraph: SparkPlanGraphCluster, | ||
exchanges: mutable.HashMap[SparkPlanInfo, SparkPlanGraphNode]): Unit = { | ||
processPlanInfo(planInfo.nodeName) match { | ||
case name if name.startsWith("WholeStageCodegen") => | ||
val metrics = planInfo.metrics.map { metric => | ||
SQLPlanMetric(metric.name, metric.accumulatorId, metric.metricType) | ||
} | ||
|
||
val cluster = new SparkPlanGraphCluster( | ||
nodeIdGenerator.getAndIncrement(), | ||
planInfo.nodeName, | ||
planInfo.simpleString, | ||
mutable.ArrayBuffer[SparkPlanGraphNode](), | ||
metrics) | ||
nodes += cluster | ||
|
||
buildSparkPlanGraphNode( | ||
planInfo.children.head, nodeIdGenerator, nodes, edges, parent, cluster, exchanges) | ||
case "InputAdapter" => | ||
buildSparkPlanGraphNode( | ||
planInfo.children.head, nodeIdGenerator, nodes, edges, parent, null, exchanges) | ||
case "BroadcastQueryStage" | "ShuffleQueryStage" => | ||
if (exchanges.contains(planInfo.children.head)) { | ||
// Point to the re-used exchange | ||
val node = exchanges(planInfo.children.head) | ||
edges += SparkPlanGraphEdge(node.id, parent.id) | ||
} else { | ||
buildSparkPlanGraphNode( | ||
planInfo.children.head, nodeIdGenerator, nodes, edges, parent, null, exchanges) | ||
} | ||
case "TableCacheQueryStage" => | ||
buildSparkPlanGraphNode( | ||
planInfo.children.head, nodeIdGenerator, nodes, edges, parent, null, exchanges) | ||
case "Subquery" | "SubqueryBroadcast" if subgraph != null => | ||
// Subquery should not be included in WholeStageCodegen | ||
buildSparkPlanGraphNode(planInfo, nodeIdGenerator, nodes, edges, parent, null, exchanges) | ||
case "Subquery" | "SubqueryBroadcast" if exchanges.contains(planInfo) => | ||
// Point to the re-used subquery | ||
val node = exchanges(planInfo) | ||
edges += SparkPlanGraphEdge(node.id, parent.id) | ||
case "ReusedSubquery" => | ||
// Re-used subquery might appear before the original subquery, so skip this node and let | ||
// the previous `case` make sure the re-used and the original point to the same node. | ||
buildSparkPlanGraphNode( | ||
planInfo.children.head, nodeIdGenerator, nodes, edges, parent, subgraph, exchanges) | ||
case "ReusedExchange" if exchanges.contains(planInfo.children.head) => | ||
// Point to the re-used exchange | ||
val node = exchanges(planInfo.children.head) | ||
edges += SparkPlanGraphEdge(node.id, parent.id) | ||
case name => | ||
val metrics = planInfo.metrics.map { metric => | ||
SQLPlanMetric(metric.name, metric.accumulatorId, metric.metricType) | ||
} | ||
val node = new SparkPlanGraphNode( | ||
nodeIdGenerator.getAndIncrement(), planInfo.nodeName, | ||
planInfo.simpleString, metrics) | ||
if (subgraph == null) { | ||
nodes += node | ||
} else { | ||
subgraph.nodes += node | ||
} | ||
if (name.contains("Exchange") || name.contains("Subquery")) { | ||
exchanges += planInfo -> node | ||
} | ||
|
||
if (parent != null) { | ||
edges += SparkPlanGraphEdge(node.id, parent.id) | ||
} | ||
planInfo.children.foreach( | ||
buildSparkPlanGraphNode(_, nodeIdGenerator, nodes, edges, node, subgraph, exchanges)) | ||
} | ||
} | ||
} |
Binary file not shown.
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