Skip to content

Commit 9a6fd1f

Browse files
author
ding
committed
mv PeriodicRDDCheckpointer to rdd.util and explain why use cache in PeriodicGraphCheckpointer
1 parent 11bc349 commit 9a6fd1f

File tree

7 files changed

+13
-8
lines changed

7 files changed

+13
-8
lines changed

core/src/main/scala/org/apache/spark/rdd/RDD.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -41,7 +41,7 @@ import org.apache.spark.partial.GroupedCountEvaluator
4141
import org.apache.spark.partial.PartialResult
4242
import org.apache.spark.storage.{RDDBlockId, StorageLevel}
4343
import org.apache.spark.util.{BoundedPriorityQueue, Utils}
44-
import org.apache.spark.util.collection.OpenHashMap
44+
import org.apache.spark.util.collection.{OpenHashMap, Utils => collectionUtils}
4545
import org.apache.spark.util.random.{BernoulliCellSampler, BernoulliSampler, PoissonSampler,
4646
SamplingUtils}
4747

@@ -1419,7 +1419,7 @@ abstract class RDD[T: ClassTag](
14191419
val mapRDDs = mapPartitions { items =>
14201420
// Priority keeps the largest elements, so let's reverse the ordering.
14211421
val queue = new BoundedPriorityQueue[T](num)(ord.reverse)
1422-
queue ++= util.collection.Utils.takeOrdered(items, num)(ord)
1422+
queue ++= collectionUtils.takeOrdered(items, num)(ord)
14231423
Iterator.single(queue)
14241424
}
14251425
if (mapRDDs.partitions.length == 0) {

core/src/main/scala/org/apache/spark/util/PeriodicRDDCheckpointer.scala renamed to core/src/main/scala/org/apache/spark/rdd/util/PeriodicRDDCheckpointer.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -15,11 +15,12 @@
1515
* limitations under the License.
1616
*/
1717

18-
package org.apache.spark.util
18+
package org.apache.spark.rdd.util
1919

2020
import org.apache.spark.SparkContext
2121
import org.apache.spark.rdd.RDD
2222
import org.apache.spark.storage.StorageLevel
23+
import org.apache.spark.util.PeriodicCheckpointer
2324

2425

2526
/**

core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -135,7 +135,7 @@ class SortingSuite extends SparkFunSuite with SharedSparkContext with Matchers w
135135
}
136136

137137
test("get a range of elements in an array not partitioned by a range partitioner") {
138-
val pairArr = util.Random.shuffle((1 to 1000).toList).map(x => (x, x))
138+
val pairArr = scala.util.Random.shuffle((1 to 1000).toList).map(x => (x, x))
139139
val pairs = sc.parallelize(pairArr, 10)
140140
val range = pairs.filterByRange(200, 800).collect()
141141
assert((800 to 200 by -1).toArray.sorted === range.map(_._1).sorted)

core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -15,13 +15,14 @@
1515
* limitations under the License.
1616
*/
1717

18-
package org.apache.spark.util
18+
package org.apache.spark.utils
1919

2020
import org.apache.hadoop.fs.Path
21-
2221
import org.apache.spark.{SharedSparkContext, SparkContext, SparkFunSuite}
2322
import org.apache.spark.rdd.RDD
23+
import org.apache.spark.rdd.util.PeriodicRDDCheckpointer
2424
import org.apache.spark.storage.StorageLevel
25+
import org.apache.spark.util.Utils
2526

2627

2728
class PeriodicRDDCheckpointerSuite extends SparkFunSuite with SharedSparkContext {

graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ import scala.reflect.ClassTag
2222
import org.apache.spark.graphx.util.PeriodicGraphCheckpointer
2323
import org.apache.spark.internal.Logging
2424
import org.apache.spark.rdd.RDD
25-
import org.apache.spark.util.PeriodicRDDCheckpointer
25+
import org.apache.spark.rdd.util.PeriodicRDDCheckpointer
2626

2727
/**
2828
* Implements a Pregel-like bulk-synchronous message-passing API.

graphx/src/main/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointer.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -87,6 +87,9 @@ private[spark] class PeriodicGraphCheckpointer[VD, ED](
8787

8888
override protected def persist(data: Graph[VD, ED]): Unit = {
8989
if (data.vertices.getStorageLevel == StorageLevel.NONE) {
90+
/* We need to use cache because persist does not honor the default storage level requested
91+
* when constructing the graph. Only cache does that.
92+
*/
9093
data.vertices.cache()
9194
}
9295
if (data.edges.getStorageLevel == StorageLevel.NONE) {

mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,8 +26,8 @@ import org.apache.spark.mllib.tree.configuration.{BoostingStrategy => OldBoostin
2626
import org.apache.spark.mllib.tree.impurity.{Variance => OldVariance}
2727
import org.apache.spark.mllib.tree.loss.{Loss => OldLoss}
2828
import org.apache.spark.rdd.RDD
29+
import org.apache.spark.rdd.util.PeriodicRDDCheckpointer
2930
import org.apache.spark.storage.StorageLevel
30-
import org.apache.spark.util.PeriodicRDDCheckpointer
3131

3232

3333
private[spark] object GradientBoostedTrees extends Logging {

0 commit comments

Comments
 (0)