@@ -27,8 +27,6 @@ import org.apache.spark.storage.StorageLevel
27
27
import org .apache .spark .graphx .impl .RoutingTablePartition
28
28
import org .apache .spark .graphx .impl .ShippableVertexPartition
29
29
import org .apache .spark .graphx .impl .VertexAttributeBlock
30
- import org .apache .spark .graphx .impl .RoutingTableMessageRDDFunctions ._
31
- import org .apache .spark .graphx .impl .VertexRDDFunctions ._
32
30
33
31
/**
34
32
* Extends `RDD[(VertexId, VD)]` by ensuring that there is only one entry for each vertex and by
@@ -233,7 +231,7 @@ class VertexRDD[@specialized VD: ClassTag](
233
231
case _ =>
234
232
this .withPartitionsRDD[VD3 ](
235
233
partitionsRDD.zipPartitions(
236
- other.copartitionWithVertices (this .partitioner.get), preservesPartitioning = true ) {
234
+ other.partitionBy (this .partitioner.get), preservesPartitioning = true ) {
237
235
(partIter, msgs) => partIter.map(_.leftJoin(msgs)(f))
238
236
}
239
237
)
@@ -277,7 +275,7 @@ class VertexRDD[@specialized VD: ClassTag](
277
275
case _ =>
278
276
this .withPartitionsRDD(
279
277
partitionsRDD.zipPartitions(
280
- other.copartitionWithVertices (this .partitioner.get), preservesPartitioning = true ) {
278
+ other.partitionBy (this .partitioner.get), preservesPartitioning = true ) {
281
279
(partIter, msgs) => partIter.map(_.innerJoin(msgs)(f))
282
280
}
283
281
)
@@ -297,7 +295,7 @@ class VertexRDD[@specialized VD: ClassTag](
297
295
*/
298
296
def aggregateUsingIndex [VD2 : ClassTag ](
299
297
messages : RDD [(VertexId , VD2 )], reduceFunc : (VD2 , VD2 ) => VD2 ): VertexRDD [VD2 ] = {
300
- val shuffled = messages.copartitionWithVertices (this .partitioner.get)
298
+ val shuffled = messages.partitionBy (this .partitioner.get)
301
299
val parts = partitionsRDD.zipPartitions(shuffled, true ) { (thisIter, msgIter) =>
302
300
thisIter.map(_.aggregateUsingIndex(msgIter, reduceFunc))
303
301
}
@@ -371,7 +369,7 @@ object VertexRDD {
371
369
def apply [VD : ClassTag ](vertices : RDD [(VertexId , VD )]): VertexRDD [VD ] = {
372
370
val vPartitioned : RDD [(VertexId , VD )] = vertices.partitioner match {
373
371
case Some (p) => vertices
374
- case None => vertices.copartitionWithVertices (new HashPartitioner (vertices.partitions.size))
372
+ case None => vertices.partitionBy (new HashPartitioner (vertices.partitions.size))
375
373
}
376
374
val vertexPartitions = vPartitioned.mapPartitions(
377
375
iter => Iterator (ShippableVertexPartition (iter)),
@@ -412,7 +410,7 @@ object VertexRDD {
412
410
): VertexRDD [VD ] = {
413
411
val vPartitioned : RDD [(VertexId , VD )] = vertices.partitioner match {
414
412
case Some (p) => vertices
415
- case None => vertices.copartitionWithVertices (new HashPartitioner (vertices.partitions.size))
413
+ case None => vertices.partitionBy (new HashPartitioner (vertices.partitions.size))
416
414
}
417
415
val routingTables = createRoutingTables(edges, vPartitioned.partitioner.get)
418
416
val vertexPartitions = vPartitioned.zipPartitions(routingTables, preservesPartitioning = true ) {
@@ -454,7 +452,7 @@ object VertexRDD {
454
452
.setName(" VertexRDD.createRoutingTables - vid2pid (aggregation)" )
455
453
456
454
val numEdgePartitions = edges.partitions.size
457
- vid2pid.copartitionWithVertices (vertexPartitioner).mapPartitions(
455
+ vid2pid.partitionBy (vertexPartitioner).mapPartitions(
458
456
iter => Iterator (RoutingTablePartition .fromMsgs(numEdgePartitions, iter)),
459
457
preservesPartitioning = true )
460
458
}
0 commit comments