|
17 | 17 |
|
18 | 18 | package org.apache.spark.scheduler |
19 | 19 |
|
20 | | -import scala.collection.mutable.ArrayBuffer |
21 | | - |
22 | | -import org.apache.spark.NarrowDependency |
23 | 20 | import org.apache.spark.annotation.DeveloperApi |
24 | | -import org.apache.spark.rdd.RDD |
25 | 21 | import org.apache.spark.storage.RDDInfo |
26 | 22 |
|
27 | 23 | /** |
@@ -54,28 +50,8 @@ private[spark] object StageInfo { |
54 | 50 | * sequence of narrow dependencies should also be associated with this Stage. |
55 | 51 | */ |
56 | 52 | def fromStage(stage: Stage): StageInfo = { |
57 | | - val ancestorRddInfos = getNarrowAncestors(stage.rdd).map(RDDInfo.fromRdd) |
| 53 | + val ancestorRddInfos = stage.rdd.getNarrowAncestors().map(RDDInfo.fromRdd) |
58 | 54 | val rddInfos = ancestorRddInfos ++ Seq(RDDInfo.fromRdd(stage.rdd)) |
59 | 55 | new StageInfo(stage.id, stage.name, stage.numTasks, rddInfos) |
60 | 56 | } |
61 | | - |
62 | | - /** |
63 | | - * Return the ancestors of the given RDD that are related to it only through a sequence of |
64 | | - * narrow dependencies. This traverses the given RDD's dependency tree using DFS. |
65 | | - */ |
66 | | - private def getNarrowAncestors( |
67 | | - rdd: RDD[_], |
68 | | - ancestors: ArrayBuffer[RDD[_]] = ArrayBuffer.empty): Seq[RDD[_]] = { |
69 | | - val narrowParents = getNarrowDependencies(rdd).map(_.rdd) |
70 | | - narrowParents.foreach { parent => |
71 | | - ancestors += parent |
72 | | - getNarrowAncestors(parent, ancestors) |
73 | | - } |
74 | | - ancestors |
75 | | - } |
76 | | - |
77 | | - /** Return the narrow dependencies of the given RDD. */ |
78 | | - private def getNarrowDependencies(rdd: RDD[_]): Seq[NarrowDependency[_]] = { |
79 | | - rdd.dependencies.collect { case d: NarrowDependency[_] => d } |
80 | | - } |
81 | 57 | } |
0 commit comments