1
+ /*
2
+ * Licensed to the Apache Software Foundation (ASF) under one or more
3
+ * contributor license agreements. See the NOTICE file distributed with
4
+ * this work for additional information regarding copyright ownership.
5
+ * The ASF licenses this file to You under the Apache License, Version 2.0
6
+ * (the "License"); you may not use this file except in compliance with
7
+ * the License. You may obtain a copy of the License at
8
+ *
9
+ * http://www.apache.org/licenses/LICENSE-2.0
10
+ *
11
+ * Unless required by applicable law or agreed to in writing, software
12
+ * distributed under the License is distributed on an "AS IS" BASIS,
13
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14
+ * See the License for the specific language governing permissions and
15
+ * limitations under the License.
16
+ */
17
+
1
18
package org .apache .spark .rdd
2
19
3
20
import java .util .concurrent .atomic .AtomicInteger
21
+
4
22
import org .apache .spark .annotation .RDDScoped
5
23
6
24
/**
7
- *
25
+ * A collection of utility methods to construct a hierarchical representation of RDD scopes.
26
+ * An RDD scope tracks the series of operations that created a given RDD.
8
27
*/
9
28
private [spark] object RDDScope {
10
29
11
- /**
12
- *
13
- */
30
+ // Symbol for delimiting each level of the hierarchy
31
+ // e.g. grandparent;parent;child
14
32
val SCOPE_NESTING_DELIMITER = " ;"
33
+
34
+ // Symbol for delimiting the scope name from the ID within each level
15
35
val SCOPE_NAME_DELIMITER = " _"
16
36
17
- /**
18
- *
19
- */
37
+ // Counter for generating scope IDs, for differentiating
38
+ // between different scopes of the same name
39
+ private val scopeCounter = new AtomicInteger (0 )
40
+
41
+ // Consider only methods that belong to these classes as potential RDD operations
42
+ // This is to limit the amount of reflection we do when we traverse the stack trace
20
43
private val classesWithScopeMethods = Set (
21
44
" org.apache.spark.SparkContext" ,
22
45
" org.apache.spark.rdd.RDD" ,
@@ -25,45 +48,61 @@ private[spark] object RDDScope {
25
48
)
26
49
27
50
/**
51
+ * Make a globally unique scope ID from the scope name.
28
52
*
29
- */
30
- private val scopeIdCounter = new AtomicInteger (0 )
31
-
32
-
33
- /**
34
- *
53
+ * For instance:
54
+ * textFile -> textFile_0
55
+ * textFile -> textFile_1
56
+ * map -> map_2
57
+ * name;with_sensitive;characters -> name-with-sensitive-characters_3
35
58
*/
36
59
private def makeScopeId (name : String ): String = {
37
60
name.replace(SCOPE_NESTING_DELIMITER , " -" ).replace(SCOPE_NAME_DELIMITER , " -" ) +
38
- SCOPE_NAME_DELIMITER + scopeIdCounter .getAndIncrement
61
+ SCOPE_NAME_DELIMITER + scopeCounter .getAndIncrement
39
62
}
40
63
41
64
/**
65
+ * Retrieve the hierarchical scope from the stack trace when an RDD is first created.
66
+ *
67
+ * This considers all methods marked with the @RDDScoped annotation and chains them together
68
+ * in the order they are invoked. Each level in the scope hierarchy represents a unique
69
+ * invocation of a particular RDD operation.
42
70
*
71
+ * For example: treeAggregate_0;reduceByKey_1;combineByKey_2;mapPartitions_3
72
+ * This means this RDD is created by the user calling treeAggregate, which calls
73
+ * `reduceByKey`, and then `combineByKey`, and then `mapPartitions` to create this RDD.
43
74
*/
44
75
private [spark] def getScope : Option [String ] = {
76
+
77
+ // TODO: Note that this approach does not correctly associate the same invocation across RDDs
78
+ // For instance, a call to `textFile` creates both a HadoopRDD and a MapPartitionsRDD, but
79
+ // there is no way to associate the invocation across these two RDDs to draw the same scope
80
+ // around them. This is because the stack trace simply does not provide information for us
81
+ // to make any reasonable association across RDDs. We may need a higher level approach that
82
+ // involves setting common variables before and after the RDD operation itself.
83
+
45
84
val rddScopeNames = Thread .currentThread.getStackTrace
46
85
// Avoid reflecting on all classes in the stack trace
47
86
.filter { ste => classesWithScopeMethods.contains(ste.getClassName) }
48
87
// Return the corresponding method if it has the @RDDScoped annotation
49
88
.flatMap { ste =>
50
- // Note that this is an approximation since we match the method only by name
51
- // Unfortunate we cannot be more precise because the stack trace does not
52
- // include parameter information
53
- Class .forName(ste.getClassName).getDeclaredMethods.find { m =>
54
- m.getName == ste.getMethodName &&
89
+ // Note that this is an approximation since we match the method only by name
90
+ // Unfortunate we cannot be more precise because the stack trace does not include
91
+ // parameter information
92
+ Class .forName(ste.getClassName).getDeclaredMethods.find { m =>
93
+ m.getName == ste.getMethodName &&
55
94
m.getDeclaredAnnotations.exists { a =>
56
95
a.annotationType() == classOf [RDDScoped ]
57
96
}
97
+ }
58
98
}
59
- }
60
99
// Use the method name as the scope name for now
61
100
.map { m => m.getName }
62
101
63
102
// It is common for such methods to internally invoke other methods with the same name
64
- // (e.g. union, reduceByKey). Here we remove adjacent duplicates such that the scope
65
- // chain does not capture this (e.g. a, a, b, c, b, c, c => a, b, c, b, c). This is
66
- // surprisingly difficult to express even in Scala.
103
+ // as aliases (e.g. union, reduceByKey). Here we remove adjacent duplicates such that
104
+ // the scope chain does not capture this (e.g. a, a, b, c, b, c, c => a, b, c, b, c).
105
+ // This is surprisingly difficult to express even in Scala.
67
106
var prev : String = null
68
107
val dedupedRddScopeNames = rddScopeNames.flatMap { n =>
69
108
if (n != prev) {
0 commit comments