Skip to content

Commit a49c2ad

Browse files
committed
[SPARK-3649] Remove GraphX custom serializers
1 parent f9d6220 commit a49c2ad

File tree

5 files changed

+6
-567
lines changed

5 files changed

+6
-567
lines changed

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

Lines changed: 6 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -27,8 +27,6 @@ import org.apache.spark.storage.StorageLevel
2727
import org.apache.spark.graphx.impl.RoutingTablePartition
2828
import org.apache.spark.graphx.impl.ShippableVertexPartition
2929
import org.apache.spark.graphx.impl.VertexAttributeBlock
30-
import org.apache.spark.graphx.impl.RoutingTableMessageRDDFunctions._
31-
import org.apache.spark.graphx.impl.VertexRDDFunctions._
3230

3331
/**
3432
* 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](
233231
case _ =>
234232
this.withPartitionsRDD[VD3](
235233
partitionsRDD.zipPartitions(
236-
other.copartitionWithVertices(this.partitioner.get), preservesPartitioning = true) {
234+
other.partitionBy(this.partitioner.get), preservesPartitioning = true) {
237235
(partIter, msgs) => partIter.map(_.leftJoin(msgs)(f))
238236
}
239237
)
@@ -277,7 +275,7 @@ class VertexRDD[@specialized VD: ClassTag](
277275
case _ =>
278276
this.withPartitionsRDD(
279277
partitionsRDD.zipPartitions(
280-
other.copartitionWithVertices(this.partitioner.get), preservesPartitioning = true) {
278+
other.partitionBy(this.partitioner.get), preservesPartitioning = true) {
281279
(partIter, msgs) => partIter.map(_.innerJoin(msgs)(f))
282280
}
283281
)
@@ -297,7 +295,7 @@ class VertexRDD[@specialized VD: ClassTag](
297295
*/
298296
def aggregateUsingIndex[VD2: ClassTag](
299297
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)
301299
val parts = partitionsRDD.zipPartitions(shuffled, true) { (thisIter, msgIter) =>
302300
thisIter.map(_.aggregateUsingIndex(msgIter, reduceFunc))
303301
}
@@ -371,7 +369,7 @@ object VertexRDD {
371369
def apply[VD: ClassTag](vertices: RDD[(VertexId, VD)]): VertexRDD[VD] = {
372370
val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match {
373371
case Some(p) => vertices
374-
case None => vertices.copartitionWithVertices(new HashPartitioner(vertices.partitions.size))
372+
case None => vertices.partitionBy(new HashPartitioner(vertices.partitions.size))
375373
}
376374
val vertexPartitions = vPartitioned.mapPartitions(
377375
iter => Iterator(ShippableVertexPartition(iter)),
@@ -412,7 +410,7 @@ object VertexRDD {
412410
): VertexRDD[VD] = {
413411
val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match {
414412
case Some(p) => vertices
415-
case None => vertices.copartitionWithVertices(new HashPartitioner(vertices.partitions.size))
413+
case None => vertices.partitionBy(new HashPartitioner(vertices.partitions.size))
416414
}
417415
val routingTables = createRoutingTables(edges, vPartitioned.partitioner.get)
418416
val vertexPartitions = vPartitioned.zipPartitions(routingTables, preservesPartitioning = true) {
@@ -454,7 +452,7 @@ object VertexRDD {
454452
.setName("VertexRDD.createRoutingTables - vid2pid (aggregation)")
455453

456454
val numEdgePartitions = edges.partitions.size
457-
vid2pid.copartitionWithVertices(vertexPartitioner).mapPartitions(
455+
vid2pid.partitionBy(vertexPartitioner).mapPartitions(
458456
iter => Iterator(RoutingTablePartition.fromMsgs(numEdgePartitions, iter)),
459457
preservesPartitioning = true)
460458
}

graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala

Lines changed: 0 additions & 50 deletions
This file was deleted.

graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala

Lines changed: 0 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -29,24 +29,6 @@ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap
2929

3030
import org.apache.spark.graphx.impl.RoutingTablePartition.RoutingTableMessage
3131

32-
private[graphx]
33-
class RoutingTableMessageRDDFunctions(self: RDD[RoutingTableMessage]) {
34-
/** Copartition an `RDD[RoutingTableMessage]` with the vertex RDD with the given `partitioner`. */
35-
def copartitionWithVertices(partitioner: Partitioner): RDD[RoutingTableMessage] = {
36-
new ShuffledRDD[VertexId, Int, Int](
37-
self, partitioner).setSerializer(new RoutingTableMessageSerializer)
38-
}
39-
}
40-
41-
private[graphx]
42-
object RoutingTableMessageRDDFunctions {
43-
import scala.language.implicitConversions
44-
45-
implicit def rdd2RoutingTableMessageRDDFunctions(rdd: RDD[RoutingTableMessage]) = {
46-
new RoutingTableMessageRDDFunctions(rdd)
47-
}
48-
}
49-
5032
private[graphx]
5133
object RoutingTablePartition {
5234
/**

0 commit comments

Comments
 (0)