diff --git a/NOTICE b/NOTICE index 063f16dc5..01612246c 100644 --- a/NOTICE +++ b/NOTICE @@ -1,4 +1,10 @@ This project uses the LDBC datasets, which are licensed under the Apache Software License, Version 2.0. -The LDBC datasets are used for testing and evaluation purposes only. +The LDBC datasets are used for testing and evaluation purposes only. The LDBC project has the following NOTICE: Note that the LDBC benchmark results should not be referred to using the words 'LDBC benchmark' or any equivalent phrase, as per the LDBC fair use policy. + +This project contains the code of Apache Spark GraphX that has the following NOTICE: +Copyright 2014-2025 The Apache Software Foundation. + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). diff --git a/build.sbt b/build.sbt index 91964728d..eaba2653c 100644 --- a/build.sbt +++ b/build.sbt @@ -72,13 +72,11 @@ publishArtifact := false lazy val commonSetting = Seq( libraryDependencies ++= Seq( - "org.apache.spark" %% "spark-graphx" % sparkVer % "provided" cross CrossVersion.for3Use2_13, "org.apache.spark" %% "spark-sql" % sparkVer % "provided" cross CrossVersion.for3Use2_13, "org.apache.spark" %% "spark-mllib" % sparkVer % "provided" cross CrossVersion.for3Use2_13, "org.slf4j" % "slf4j-api" % "2.0.17" % "provided", "org.scalatest" %% "scalatest" % defaultScalaTestVer % Test, "com.github.zafarkhaja" % "java-semver" % "0.10.2" % Test), - Compile / scalacOptions ++= Seq("-deprecation", "-feature"), Compile / doc / scalacOptions ++= Seq( "-groups", "-implicits", @@ -114,9 +112,43 @@ lazy val commonSetting = Seq( ScalacOptions.source3, ScalacOptions.fatalWarnings), tpolecatExcludeOptions ++= Set(ScalacOptions.warnNonUnitStatement), - Test / tpolecatExcludeOptions ++= Set(ScalacOptions.warnValueDiscard)) + Test / tpolecatExcludeOptions ++= Set( + ScalacOptions.warnValueDiscard, + ScalacOptions.warnUnusedLocals, + ScalacOptions.warnUnusedExplicits, + ScalacOptions.warnUnusedImplicits, + ScalacOptions.warnUnusedParams, + ScalacOptions.warnUnusedPrivates, + ScalacOptions.warnNumericWiden, + ScalacOptions.privateWarnNumericWiden, + )) + +lazy val graphx = (project in file("graphx")) + .settings( + commonSetting, + name := "graphframes-graphx", + moduleName := s"${name.value}-spark$sparkMajorVer", + // Export the JAR so that this can be excluded from shading in connect + exportJars := true, + + // for scala 2.13 we should mark "unused" class tags by @nowarn, + // for scala 2.12 we shouldn't + // the only way at the moment is to not check unused @nowarn for GraphX + tpolecatExcludeOptions ++= Set(ScalacOptions.warnUnusedNoWarn, ScalacOptions.privateWarnUnusedNoWarn), + + // Global settings + Global / concurrentRestrictions := Seq(Tags.limitAll(1)), + autoAPIMappings := true, + coverageHighlighting := false, + Test / packageBin / publishArtifact := false, + Test / packageDoc / publishArtifact := false, + Test / packageSrc / publishArtifact := false, + Compile / packageBin / publishArtifact := true, + Compile / packageDoc / publishArtifact := true, + Compile / packageSrc / publishArtifact := true) lazy val core = (project in file("core")) + .dependsOn(graphx) .settings( commonSetting, name := "graphframes", @@ -217,8 +249,8 @@ lazy val docs = (project in file("docs")) .withConfigValue(LaikaKeys.siteBaseURL, siteBaseUri) .withConfigValue("pydoc.baseUri", s"$siteBaseUri/api/python") .withConfigValue("scaladoc.baseUri", s"$siteBaseUri/api/scaladoc") - .withConfigValue("spark.version", sparkVer) - .withConfigValue("scala.version", scalaVer), + .withConfigValue("spark.version", sparkVer) + .withConfigValue("scala.version", scalaVer), laikaExtensions := Seq(GitHubFlavor, SyntaxHighlighting, LaikaCustomDirectives), laikaHTML := (laikaHTML dependsOn mdoc.toTask( "") dependsOn generateAtomFeed dependsOn buildAndCopyScalaDoc dependsOn buildAndCopyPythonDoc dependsOn (core / Compile / doc)).value, diff --git a/core/src/main/scala/org/graphframes/GraphFrame.scala b/core/src/main/scala/org/graphframes/GraphFrame.scala index f3bfa6029..f39c9343d 100644 --- a/core/src/main/scala/org/graphframes/GraphFrame.scala +++ b/core/src/main/scala/org/graphframes/GraphFrame.scala @@ -17,8 +17,8 @@ package org.graphframes -import org.apache.spark.graphx.Edge -import org.apache.spark.graphx.Graph +import org.apache.spark.graphframes.graphx.Edge +import org.apache.spark.graphframes.graphx.Graph import org.apache.spark.ml.clustering.PowerIterationClustering import org.apache.spark.sql._ import org.apache.spark.sql.functions.array diff --git a/core/src/main/scala/org/graphframes/examples/BeliefPropagation.scala b/core/src/main/scala/org/graphframes/examples/BeliefPropagation.scala index dfe8e7257..0ed3c6698 100644 --- a/core/src/main/scala/org/graphframes/examples/BeliefPropagation.scala +++ b/core/src/main/scala/org/graphframes/examples/BeliefPropagation.scala @@ -17,9 +17,9 @@ package org.graphframes.examples -import org.apache.spark.graphx.Graph -import org.apache.spark.graphx.VertexRDD -import org.apache.spark.graphx.{Edge => GXEdge} +import org.apache.spark.graphframes.graphx.Graph +import org.apache.spark.graphframes.graphx.VertexRDD +import org.apache.spark.graphframes.graphx.{Edge => GXEdge} import org.apache.spark.sql.Column import org.apache.spark.sql.Row import org.apache.spark.sql.SparkSession diff --git a/core/src/main/scala/org/graphframes/lib/ConnectedComponents.scala b/core/src/main/scala/org/graphframes/lib/ConnectedComponents.scala index fa7c7c335..11ef58b1c 100644 --- a/core/src/main/scala/org/graphframes/lib/ConnectedComponents.scala +++ b/core/src/main/scala/org/graphframes/lib/ConnectedComponents.scala @@ -18,6 +18,7 @@ package org.graphframes.lib import org.apache.hadoop.fs.Path +import org.apache.spark.graphframes.graphx import org.apache.spark.sql.Column import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ @@ -183,7 +184,7 @@ object ConnectedComponents extends Logging { private def runGraphX(graph: GraphFrame, maxIter: Int): DataFrame = { val components = - org.apache.spark.graphx.lib.ConnectedComponents.run(graph.cachedTopologyGraphX, maxIter) + graphx.lib.ConnectedComponents.run(graph.cachedTopologyGraphX, maxIter) GraphXConversions.fromGraphX(graph, components, vertexNames = Seq(COMPONENT)).vertices } diff --git a/core/src/main/scala/org/graphframes/lib/GraphXConversions.scala b/core/src/main/scala/org/graphframes/lib/GraphXConversions.scala index 4cd284e1e..e356b1bbc 100644 --- a/core/src/main/scala/org/graphframes/lib/GraphXConversions.scala +++ b/core/src/main/scala/org/graphframes/lib/GraphXConversions.scala @@ -17,7 +17,7 @@ package org.graphframes.lib -import org.apache.spark.graphx.Graph +import org.apache.spark.graphframes.graphx.Graph import org.apache.spark.sql.DataFrame import org.apache.spark.sql.Row import org.apache.spark.sql.functions._ diff --git a/core/src/main/scala/org/graphframes/lib/LabelPropagation.scala b/core/src/main/scala/org/graphframes/lib/LabelPropagation.scala index 495942ec4..f17b82420 100644 --- a/core/src/main/scala/org/graphframes/lib/LabelPropagation.scala +++ b/core/src/main/scala/org/graphframes/lib/LabelPropagation.scala @@ -17,7 +17,7 @@ package org.graphframes.lib -import org.apache.spark.graphx.{lib => graphxlib} +import org.apache.spark.graphframes.graphx.{lib => graphxlib} import org.apache.spark.sql.Column import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ diff --git a/core/src/main/scala/org/graphframes/lib/PageRank.scala b/core/src/main/scala/org/graphframes/lib/PageRank.scala index 44e78b3e5..b0c851264 100644 --- a/core/src/main/scala/org/graphframes/lib/PageRank.scala +++ b/core/src/main/scala/org/graphframes/lib/PageRank.scala @@ -17,7 +17,7 @@ package org.graphframes.lib -import org.apache.spark.graphx.{lib => graphxlib} +import org.apache.spark.graphframes.graphx.{lib => graphxlib} import org.graphframes.GraphFrame /** diff --git a/core/src/main/scala/org/graphframes/lib/ParallelPersonalizedPageRank.scala b/core/src/main/scala/org/graphframes/lib/ParallelPersonalizedPageRank.scala index 1632485d9..7c8fea43a 100644 --- a/core/src/main/scala/org/graphframes/lib/ParallelPersonalizedPageRank.scala +++ b/core/src/main/scala/org/graphframes/lib/ParallelPersonalizedPageRank.scala @@ -17,7 +17,7 @@ package org.graphframes.lib -import org.apache.spark.graphx.{lib => graphxlib} +import org.apache.spark.graphframes.graphx.{lib => graphxlib} import org.graphframes.GraphFrame import org.graphframes.WithMaxIter diff --git a/core/src/main/scala/org/graphframes/lib/SVDPlusPlus.scala b/core/src/main/scala/org/graphframes/lib/SVDPlusPlus.scala index 61847faf4..093f8f0d8 100644 --- a/core/src/main/scala/org/graphframes/lib/SVDPlusPlus.scala +++ b/core/src/main/scala/org/graphframes/lib/SVDPlusPlus.scala @@ -17,8 +17,8 @@ package org.graphframes.lib -import org.apache.spark.graphx.Edge -import org.apache.spark.graphx.{lib => graphxlib} +import org.apache.spark.graphframes.graphx.Edge +import org.apache.spark.graphframes.graphx.{lib => graphxlib} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.Row import org.graphframes.GraphFrame diff --git a/core/src/main/scala/org/graphframes/lib/ShortestPaths.scala b/core/src/main/scala/org/graphframes/lib/ShortestPaths.scala index 50a543cc8..cc972d8fb 100644 --- a/core/src/main/scala/org/graphframes/lib/ShortestPaths.scala +++ b/core/src/main/scala/org/graphframes/lib/ShortestPaths.scala @@ -17,7 +17,7 @@ package org.graphframes.lib -import org.apache.spark.graphx.{lib => graphxlib} +import org.apache.spark.graphframes.graphx.{lib => graphxlib} import org.apache.spark.sql.Column import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions.col diff --git a/core/src/main/scala/org/graphframes/lib/StronglyConnectedComponents.scala b/core/src/main/scala/org/graphframes/lib/StronglyConnectedComponents.scala index 179e3d316..0f52ae7ca 100644 --- a/core/src/main/scala/org/graphframes/lib/StronglyConnectedComponents.scala +++ b/core/src/main/scala/org/graphframes/lib/StronglyConnectedComponents.scala @@ -17,7 +17,7 @@ package org.graphframes.lib -import org.apache.spark.graphx.{lib => graphxlib} +import org.apache.spark.graphframes.graphx.{lib => graphxlib} import org.apache.spark.sql.DataFrame import org.graphframes.GraphFrame import org.graphframes.WithMaxIter diff --git a/core/src/test/scala/org/graphframes/GraphFrameSuite.scala b/core/src/test/scala/org/graphframes/GraphFrameSuite.scala index 7477a5603..ca9861ff9 100644 --- a/core/src/test/scala/org/graphframes/GraphFrameSuite.scala +++ b/core/src/test/scala/org/graphframes/GraphFrameSuite.scala @@ -19,8 +19,8 @@ package org.graphframes import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.Path -import org.apache.spark.graphx.Edge -import org.apache.spark.graphx.Graph +import org.apache.spark.graphframes.graphx.Edge +import org.apache.spark.graphframes.graphx.Graph import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame import org.apache.spark.sql.Row diff --git a/core/src/test/scala/org/graphframes/SparkFunSuite.scala b/core/src/test/scala/org/graphframes/SparkFunSuite.scala index 7150faad9..d48b65498 100644 --- a/core/src/test/scala/org/graphframes/SparkFunSuite.scala +++ b/core/src/test/scala/org/graphframes/SparkFunSuite.scala @@ -42,5 +42,4 @@ private[graphframes] abstract class SparkFunSuite extends AnyFunSuite with Loggi logInfo(s"\n\n===== FINISHED $shortSuiteName: '$testName' =====\n") } } - } diff --git a/graphx/src/main/java/org/apache/spark/graphframes/graphx/TripletFields.java b/graphx/src/main/java/org/apache/spark/graphframes/graphx/TripletFields.java new file mode 100644 index 000000000..c9c52e9dc --- /dev/null +++ b/graphx/src/main/java/org/apache/spark/graphframes/graphx/TripletFields.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx; + +import java.io.Serializable; + +/** + * Represents a subset of the fields of an [[EdgeTriplet]] or [[EdgeContext]]. This allows the + * system to populate only those fields for efficiency. + */ +public class TripletFields implements Serializable { + + /** Indicates whether the source vertex attribute is included. */ + public final boolean useSrc; + + /** Indicates whether the destination vertex attribute is included. */ + public final boolean useDst; + + /** Indicates whether the edge attribute is included. */ + public final boolean useEdge; + + /** Constructs a default TripletFields in which all fields are included. */ + public TripletFields() { + this(true, true, true); + } + + public TripletFields(boolean useSrc, boolean useDst, boolean useEdge) { + this.useSrc = useSrc; + this.useDst = useDst; + this.useEdge = useEdge; + } + + /** + * None of the triplet fields are exposed. + */ + public static final TripletFields None = new TripletFields(false, false, false); + + /** + * Expose only the edge field and not the source or destination field. + */ + public static final TripletFields EdgeOnly = new TripletFields(false, false, true); + + /** + * Expose the source and edge fields but not the destination field. (Same as Src) + */ + public static final TripletFields Src = new TripletFields(true, false, true); + + /** + * Expose the destination and edge fields but not the source field. (Same as Dst) + */ + public static final TripletFields Dst = new TripletFields(false, true, true); + + /** + * Expose all the fields (source, edge, and destination). + */ + public static final TripletFields All = new TripletFields(true, true, true); +} diff --git a/graphx/src/main/java/org/apache/spark/graphframes/graphx/impl/EdgeActiveness.java b/graphx/src/main/java/org/apache/spark/graphframes/graphx/impl/EdgeActiveness.java new file mode 100644 index 000000000..8951180af --- /dev/null +++ b/graphx/src/main/java/org/apache/spark/graphframes/graphx/impl/EdgeActiveness.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.impl; + +/** + * Criteria for filtering edges based on activeness. For internal use only. + */ +public enum EdgeActiveness { + /** Neither the source vertex nor the destination vertex need be active. */ + Neither, + /** The source vertex must be active. */ + SrcOnly, + /** The destination vertex must be active. */ + DstOnly, + /** Both vertices must be active. */ + Both, + /** At least one vertex must be active. */ + Either +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/Edge.scala new file mode 100644 index 000000000..9333bfe84 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/Edge.scala @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.apache.spark.util.collection.SortDataFormat + +/** + * A single directed edge consisting of a source id, target id, and the data associated with the + * edge. + * + * @tparam ED + * type of the edge attribute + * + * @param srcId + * The vertex id of the source vertex + * @param dstId + * The vertex id of the target vertex + * @param attr + * The attribute associated with the edge + */ +case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED]( + var srcId: VertexId = 0, + var dstId: VertexId = 0, + var attr: ED = null.asInstanceOf[ED]) + extends Serializable { + + /** + * Given one vertex in the edge return the other vertex. + * + * @param vid + * the id one of the two vertices on the edge. + * @return + * the id of the other vertex on the edge. + */ + def otherVertexId(vid: VertexId): VertexId = + if (srcId == vid) dstId else { assert(dstId == vid); srcId } + + /** + * Return the relative direction of the edge to the corresponding vertex. + * + * @param vid + * the id of one of the two vertices in the edge. + * @return + * the relative direction of the edge to the corresponding vertex. + */ + def relativeDirection(vid: VertexId): EdgeDirection = + if (vid == srcId) EdgeDirection.Out else { assert(vid == dstId); EdgeDirection.In } +} + +object Edge { + private[graphx] def lexicographicOrdering[ED] = new Ordering[Edge[ED]] { + override def compare(a: Edge[ED], b: Edge[ED]): Int = { + if (a.srcId == b.srcId) { + if (a.dstId == b.dstId) 0 + else if (a.dstId < b.dstId) -1 + else 1 + } else if (a.srcId < b.srcId) -1 + else 1 + } + } + + private[graphx] def edgeArraySortDataFormat[ED] = + new SortDataFormat[Edge[ED], Array[Edge[ED]]] { + override def getKey(data: Array[Edge[ED]], pos: Int): Edge[ED] = { + data(pos) + } + + override def swap(data: Array[Edge[ED]], pos0: Int, pos1: Int): Unit = { + val tmp = data(pos0) + data(pos0) = data(pos1) + data(pos1) = tmp + } + + override def copyElement( + src: Array[Edge[ED]], + srcPos: Int, + dst: Array[Edge[ED]], + dstPos: Int): Unit = { + dst(dstPos) = src(srcPos) + } + + override def copyRange( + src: Array[Edge[ED]], + srcPos: Int, + dst: Array[Edge[ED]], + dstPos: Int, + length: Int): Unit = { + System.arraycopy(src, srcPos, dst, dstPos, length) + } + + override def allocate(length: Int): Array[Edge[ED]] = { + new Array[Edge[ED]](length) + } + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/EdgeContext.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/EdgeContext.scala new file mode 100644 index 000000000..4d7a1282b --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/EdgeContext.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +/** + * Represents an edge along with its neighboring vertices and allows sending messages along the + * edge. Used in [[Graph#aggregateMessages]]. + */ +abstract class EdgeContext[VD, ED, A] { + + /** The vertex id of the edge's source vertex. */ + def srcId: VertexId + + /** The vertex id of the edge's destination vertex. */ + def dstId: VertexId + + /** The vertex attribute of the edge's source vertex. */ + def srcAttr: VD + + /** The vertex attribute of the edge's destination vertex. */ + def dstAttr: VD + + /** The attribute associated with the edge. */ + def attr: ED + + /** Sends a message to the source vertex. */ + def sendToSrc(msg: A): Unit + + /** Sends a message to the destination vertex. */ + def sendToDst(msg: A): Unit + + /** Converts the edge and vertex properties into an [[EdgeTriplet]] for convenience. */ + def toEdgeTriplet: EdgeTriplet[VD, ED] = { + val et = new EdgeTriplet[VD, ED] + et.srcId = srcId + et.srcAttr = srcAttr + et.dstId = dstId + et.dstAttr = dstAttr + et.attr = attr + et + } +} + +object EdgeContext { + + /** + * Extractor mainly used for Graph#aggregateMessages*. Example: + * {{{ + * val messages = graph.aggregateMessages( + * case ctx @ EdgeContext(_, _, _, _, attr) => + * ctx.sendToDst(attr) + * , _ + _) + * }}} + */ + def unapply[VD, ED, A](edge: EdgeContext[VD, ED, A]): Some[(VertexId, VertexId, VD, VD, ED)] = + Some((edge.srcId, edge.dstId, edge.srcAttr, edge.dstAttr, edge.attr)) +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/EdgeDirection.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/EdgeDirection.scala new file mode 100644 index 000000000..104b7bd25 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/EdgeDirection.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +/** + * The direction of a directed edge relative to a vertex. + */ +class EdgeDirection private (private val name: String) extends Serializable { + + /** + * Reverse the direction of an edge. An in becomes out, out becomes in and both and either + * remain the same. + */ + def reverse: EdgeDirection = this match { + case EdgeDirection.In => EdgeDirection.Out + case EdgeDirection.Out => EdgeDirection.In + case EdgeDirection.Either => EdgeDirection.Either + case EdgeDirection.Both => EdgeDirection.Both + // unreachable + case _ => throw new IllegalArgumentException(s"Unknown edge direction: $this") + } + + override def toString: String = "EdgeDirection." + name + + override def equals(o: Any): Boolean = o match { + case other: EdgeDirection => other.name == name + case _ => false + } + + override def hashCode: Int = name.hashCode +} + +/** + * A set of [[EdgeDirection]]s. + */ +object EdgeDirection { + + /** Edges arriving at a vertex. */ + final val In: EdgeDirection = new EdgeDirection("In") + + /** Edges originating from a vertex. */ + final val Out: EdgeDirection = new EdgeDirection("Out") + + /** Edges originating from *or* arriving at a vertex of interest. */ + final val Either: EdgeDirection = new EdgeDirection("Either") + + /** Edges originating from *and* arriving at a vertex of interest. */ + final val Both: EdgeDirection = new EdgeDirection("Both") +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/EdgeRDD.scala new file mode 100644 index 000000000..210c3b204 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/EdgeRDD.scala @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.apache.spark.Dependency +import org.apache.spark.Partition +import org.apache.spark.SparkContext +import org.apache.spark.TaskContext +import org.apache.spark.graphframes.graphx.impl.EdgePartition +import org.apache.spark.graphframes.graphx.impl.EdgePartitionBuilder +import org.apache.spark.graphframes.graphx.impl.EdgeRDDImpl +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + +import scala.reflect.ClassTag + +/** + * `EdgeRDD[ED, VD]` extends `RDD[Edge[ED]]` by storing the edges in columnar format on each + * partition for performance. It may additionally store the vertex attributes associated with each + * edge to provide the triplet view. Shipping of the vertex attributes is managed by + * `impl.ReplicatedVertexView`. + */ +abstract class EdgeRDD[ED](sc: SparkContext, deps: Seq[Dependency[_]]) + extends RDD[Edge[ED]](sc, deps) { + + // scalastyle:off structural.type + private[graphx] def partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])] forSome { type VD } + // scalastyle:on structural.type + + override protected def getPartitions: Array[Partition] = partitionsRDD.partitions + + override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = { + val p = firstParent[(PartitionID, EdgePartition[ED, _])].iterator(part, context) + if (p.hasNext) { + p.next()._2.iterator.map(_.copy()) + } else { + Iterator.empty + } + } + + /** + * Map the values in an edge partitioning preserving the structure but changing the values. + * + * @tparam ED2 + * the new edge value type + * @param f + * the function from an edge to a new edge value + * @return + * a new EdgeRDD containing the new edge values + */ + def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDD[ED2] + + /** + * Reverse all the edges in this RDD. + * + * @return + * a new EdgeRDD containing all the edges reversed + */ + def reverse: EdgeRDD[ED] + + /** + * Inner joins this EdgeRDD with another EdgeRDD, assuming both are partitioned using the same + * [[PartitionStrategy]]. + * + * @param other + * the EdgeRDD to join with + * @param f + * the join function applied to corresponding values of `this` and `other` + * @return + * a new EdgeRDD containing only edges that appear in both `this` and `other`, with values + * supplied by `f` + */ + def innerJoin[ED2: ClassTag, ED3: ClassTag](other: EdgeRDD[ED2])( + f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3] + + /** + * Changes the target storage level while preserving all other properties of the EdgeRDD. + * Operations on the returned EdgeRDD will preserve this storage level. + * + * This does not actually trigger a cache; to do this, call + * [[org.apache.spark.graphframes.graphx.EdgeRDD#cache]] on the returned EdgeRDD. + */ + private[graphx] def withTargetStorageLevel(targetStorageLevel: StorageLevel): EdgeRDD[ED] +} + +object EdgeRDD { + + /** + * Creates an EdgeRDD from a set of edges. + * + * @tparam ED + * the edge attribute type + * @tparam VD + * the type of the vertex attributes that may be joined with the returned EdgeRDD + */ + def fromEdges[ED: ClassTag, VD: ClassTag](edges: RDD[Edge[ED]]): EdgeRDDImpl[ED, VD] = { + val edgePartitions = edges.mapPartitionsWithIndex { (pid, iter) => + val builder = new EdgePartitionBuilder[ED, VD] + iter.foreach { e => + builder.add(e.srcId, e.dstId, e.attr) + } + Iterator((pid, builder.toEdgePartition)) + } + EdgeRDD.fromEdgePartitions(edgePartitions) + } + + /** + * Creates an EdgeRDD from already-constructed edge partitions. + * + * @tparam ED + * the edge attribute type + * @tparam VD + * the type of the vertex attributes that may be joined with the returned EdgeRDD + */ + private[graphx] def fromEdgePartitions[ED: ClassTag, VD: ClassTag]( + edgePartitions: RDD[(Int, EdgePartition[ED, VD])]): EdgeRDDImpl[ED, VD] = { + new EdgeRDDImpl(edgePartitions) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/EdgeTriplet.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/EdgeTriplet.scala new file mode 100644 index 000000000..37661b8e3 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/EdgeTriplet.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import scala.annotation.nowarn + +/** + * An edge triplet represents an edge along with the vertex attributes of its neighboring + * vertices. + * + * @tparam VD + * the type of the vertex attribute. + * @tparam ED + * the type of the edge attribute + */ +@nowarn +class EdgeTriplet[VD, ED] extends Edge[ED] { + + /** + * The source vertex attribute + */ + var srcAttr: VD = _ // nullValue[VD] + + /** + * The destination vertex attribute + */ + var dstAttr: VD = _ // nullValue[VD] + + /** + * Set the edge properties of this triplet. + */ + protected[spark] def set(other: Edge[ED]): EdgeTriplet[VD, ED] = { + srcId = other.srcId + dstId = other.dstId + attr = other.attr + this + } + + /** + * Given one vertex in the edge return the other vertex. + * + * @param vid + * the id one of the two vertices on the edge + * @return + * the attribute for the other vertex on the edge + */ + def otherVertexAttr(vid: VertexId): VD = + if (srcId == vid) dstAttr else { assert(dstId == vid); srcAttr } + + /** + * Get the vertex object for the given vertex in the edge. + * + * @param vid + * the id of one of the two vertices on the edge + * @return + * the attr for the vertex with that id + */ + def vertexAttr(vid: VertexId): VD = + if (srcId == vid) srcAttr else { assert(dstId == vid); dstAttr } + + override def toString: String = ((srcId, srcAttr), (dstId, dstAttr), attr).toString() + + def toTuple: ((VertexId, VD), (VertexId, VD), ED) = ((srcId, srcAttr), (dstId, dstAttr), attr) +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/Graph.scala new file mode 100644 index 000000000..b57b20f91 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/Graph.scala @@ -0,0 +1,591 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.apache.spark.graphframes.graphx.impl._ +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + +import scala.annotation.nowarn +import scala.reflect.ClassTag + +/** + * The Graph abstractly represents a graph with arbitrary objects associated with vertices and + * edges. The graph provides basic operations to access and manipulate the data associated with + * vertices and edges as well as the underlying structure. Like Spark RDDs, the graph is a + * functional data-structure in which mutating operations return new graphs. + * + * @note + * [[GraphOps]] contains additional convenience operations and graph algorithms. + * + * @tparam VD + * the vertex attribute type + * @tparam ED + * the edge attribute type + */ +abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializable { + + /** + * An RDD containing the vertices and their associated attributes. + * + * @note + * vertex ids are unique. + * @return + * an RDD containing the vertices in this graph + */ + val vertices: VertexRDD[VD] + + /** + * An RDD containing the edges and their associated attributes. The entries in the RDD contain + * just the source id and target id along with the edge data. + * + * @return + * an RDD containing the edges in this graph + * + * @see + * `Edge` for the edge type. + * @see + * `Graph#triplets` to get an RDD which contains all the edges along with their vertex data. + */ + val edges: EdgeRDD[ED] + + /** + * An RDD containing the edge triplets, which are edges along with the vertex data associated + * with the adjacent vertices. The caller should use [[edges]] if the vertex data are not + * needed, i.e. if only the edge data and adjacent vertex ids are needed. + * + * @return + * an RDD containing edge triplets + * + * @example + * This operation might be used to evaluate a graph coloring where we would like to check that + * both vertices are a different color. + * {{{ + * type Color = Int + * val graph: Graph[Color, Int] = GraphLoader.edgeListFile("hdfs://file.tsv") + * val numInvalid = graph.triplets.map(e => if (e.src.data == e.dst.data) 1 else 0).sum + * }}} + */ + val triplets: RDD[EdgeTriplet[VD, ED]] + + /** + * Caches the vertices and edges associated with this graph at the specified storage level, + * ignoring any target storage levels previously set. + * + * @param newLevel + * the level at which to cache the graph. + * + * @return + * A reference to this graph for convenience. + */ + def persist(newLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED] + + /** + * Caches the vertices and edges associated with this graph at the previously-specified target + * storage levels, which default to `MEMORY_ONLY`. This is used to pin a graph in memory + * enabling multiple queries to reuse the same construction process. + */ + def cache(): Graph[VD, ED] + + /** + * Mark this Graph for checkpointing. It will be saved to a file inside the checkpoint directory + * set with SparkContext.setCheckpointDir() and all references to its parent RDDs will be + * removed. It is strongly recommended that this Graph is persisted in memory, otherwise saving + * it on a file will require recomputation. + */ + def checkpoint(): Unit + + /** + * Return whether this Graph has been checkpointed or not. This returns true iff both the + * vertices RDD and edges RDD have been checkpointed. + */ + def isCheckpointed: Boolean + + /** + * Gets the name of the files to which this Graph was checkpointed. (The vertices RDD and edges + * RDD are checkpointed separately.) + */ + def getCheckpointFiles: Seq[String] + + /** + * Uncaches both vertices and edges of this graph. This is useful in iterative algorithms that + * build a new graph in each iteration. + * + * @param blocking + * Whether to block until all data is unpersisted (default: false) + */ + def unpersist(blocking: Boolean = false): Graph[VD, ED] + + /** + * Uncaches only the vertices of this graph, leaving the edges alone. This is useful in + * iterative algorithms that modify the vertex attributes but reuse the edges. This method can + * be used to uncache the vertex attributes of previous iterations once they are no longer + * needed, improving GC performance. + * + * @param blocking + * Whether to block until all data is unpersisted (default: false) + */ + def unpersistVertices(blocking: Boolean = false): Graph[VD, ED] + + /** + * Repartitions the edges in the graph according to `partitionStrategy`. + * + * @param partitionStrategy + * the partitioning strategy to use when partitioning the edges in the graph. + */ + def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] + + /** + * Repartitions the edges in the graph according to `partitionStrategy`. + * + * @param partitionStrategy + * the partitioning strategy to use when partitioning the edges in the graph. + * @param numPartitions + * the number of edge partitions in the new graph. + */ + def partitionBy(partitionStrategy: PartitionStrategy, numPartitions: Int): Graph[VD, ED] + + /** + * Transforms each vertex attribute in the graph using the map function. + * + * @note + * The new graph has the same structure. As a consequence the underlying index structures can + * be reused. + * + * @param map + * the function from a vertex object to a new vertex value + * + * @tparam VD2 + * the new vertex data type + * + * @example + * We might use this operation to change the vertex values from one type to another to + * initialize an algorithm. + * {{{ + * val rawGraph: Graph[(), ()] = Graph.textFile("hdfs://file") + * val root = 42 + * var bfsGraph = rawGraph.mapVertices[Int]((vid, data) => if (vid == root) 0 else Math.MaxValue) + * }}} + */ + def mapVertices[VD2: ClassTag](map: (VertexId, VD) => VD2)(implicit + eq: VD =:= VD2 = null): Graph[VD2, ED] + + /** + * Transforms each edge attribute in the graph using the map function. The map function is not + * passed the vertex value for the vertices adjacent to the edge. If vertex values are desired, + * use `mapTriplets`. + * + * @note + * This graph is not changed and that the new graph has the same structure. As a consequence + * the underlying index structures can be reused. + * + * @param map + * the function from an edge object to a new edge value. + * + * @tparam ED2 + * the new edge data type + * + * @example + * This function might be used to initialize edge attributes. + */ + def mapEdges[ED2: ClassTag](map: Edge[ED] => ED2): Graph[VD, ED2] = { + mapEdges((_, iter) => iter.map(map)) + } + + /** + * Transforms each edge attribute using the map function, passing it a whole partition at a + * time. The map function is given an iterator over edges within a logical partition as well as + * the partition's ID, and it should return a new iterator over the new values of each edge. The + * new iterator's elements must correspond one-to-one with the old iterator's elements. If + * adjacent vertex values are desired, use `mapTriplets`. + * + * @note + * This does not change the structure of the graph or modify the values of this graph. As a + * consequence the underlying index structures can be reused. + * + * @param map + * a function that takes a partition id and an iterator over all the edges in the partition, + * and must return an iterator over the new values for each edge in the order of the input + * iterator + * + * @tparam ED2 + * the new edge data type + */ + def mapEdges[ED2: ClassTag]( + map: (PartitionID, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] + + /** + * Transforms each edge attribute using the map function, passing it the adjacent vertex + * attributes as well. If adjacent vertex values are not required, consider using `mapEdges` + * instead. + * + * @note + * This does not change the structure of the graph or modify the values of this graph. As a + * consequence the underlying index structures can be reused. + * + * @param map + * the function from an edge object to a new edge value. + * + * @tparam ED2 + * the new edge data type + * + * @example + * This function might be used to initialize edge attributes based on the attributes + * associated with each vertex. + * {{{ + * val rawGraph: Graph[Int, Int] = someLoadFunction() + * val graph = rawGraph.mapTriplets[Int]( edge => + * edge.src.data - edge.dst.data) + * }}} + */ + def mapTriplets[ED2: ClassTag](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { + mapTriplets((_, iter) => iter.map(map), TripletFields.All) + } + + /** + * Transforms each edge attribute using the map function, passing it the adjacent vertex + * attributes as well. If adjacent vertex values are not required, consider using `mapEdges` + * instead. + * + * @note + * This does not change the structure of the graph or modify the values of this graph. As a + * consequence the underlying index structures can be reused. + * + * @param map + * the function from an edge object to a new edge value. + * @param tripletFields + * which fields should be included in the edge triplet passed to the map function. If not all + * fields are needed, specifying this can improve performance. + * + * @tparam ED2 + * the new edge data type + * + * @example + * This function might be used to initialize edge attributes based on the attributes + * associated with each vertex. + * {{{ + * val rawGraph: Graph[Int, Int] = someLoadFunction() + * val graph = rawGraph.mapTriplets[Int]( edge => + * edge.src.data - edge.dst.data) + * }}} + */ + def mapTriplets[ED2: ClassTag]( + map: EdgeTriplet[VD, ED] => ED2, + tripletFields: TripletFields): Graph[VD, ED2] = { + mapTriplets((_, iter) => iter.map(map), tripletFields) + } + + /** + * Transforms each edge attribute a partition at a time using the map function, passing it the + * adjacent vertex attributes as well. The map function is given an iterator over edge triplets + * within a logical partition and should yield a new iterator over the new values of each edge + * in the order in which they are provided. If adjacent vertex values are not required, consider + * using `mapEdges` instead. + * + * @note + * This does not change the structure of the graph or modify the values of this graph. As a + * consequence the underlying index structures can be reused. + * + * @param map + * the iterator transform + * @param tripletFields + * which fields should be included in the edge triplet passed to the map function. If not all + * fields are needed, specifying this can improve performance. + * + * @tparam ED2 + * the new edge data type + */ + def mapTriplets[ED2: ClassTag]( + map: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2], + tripletFields: TripletFields): Graph[VD, ED2] + + /** + * Reverses all edges in the graph. If this graph contains an edge from a to b then the returned + * graph contains an edge from b to a. + */ + def reverse: Graph[VD, ED] + + /** + * Restricts the graph to only the vertices and edges satisfying the predicates. The resulting + * subgraph satisfies + * + * {{{ + * V' = {v : for all v in V where vpred(v)} + * E' = {(u,v): for all (u,v) in E where epred((u,v)) && vpred(u) && vpred(v)} + * }}} + * + * @param epred + * the edge predicate, which takes a triplet and evaluates to true if the edge is to remain in + * the subgraph. Note that only edges where both vertices satisfy the vertex predicate are + * considered. + * + * @param vpred + * the vertex predicate, which takes a vertex object and evaluates to true if the vertex is to + * be included in the subgraph + * + * @return + * the subgraph containing only the vertices and edges that satisfy the predicates + */ + def subgraph( + epred: EdgeTriplet[VD, ED] => Boolean = _ => true, + vpred: (VertexId, VD) => Boolean = (_, _) => true): Graph[VD, ED] + + /** + * Restricts the graph to only the vertices and edges that are also in `other`, but keeps the + * attributes from this graph. + * @param other + * the graph to project this graph onto + * @return + * a graph with vertices and edges that exist in both the current graph and `other`, with + * vertex and edge data from the current graph + */ + def mask[VD2: ClassTag, ED2: ClassTag](other: Graph[VD2, ED2]): Graph[VD, ED] + + /** + * Merges multiple edges between two vertices into a single edge. For correct results, the graph + * must have been partitioned using `partitionBy`. + * + * @param merge + * the user-supplied commutative associative function to merge edge attributes for duplicate + * edges. + * + * @return + * The resulting graph with a single edge for each (source, dest) vertex pair. + */ + def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] + + /** + * Aggregates values from the neighboring edges and vertices of each vertex. The user-supplied + * `sendMsg` function is invoked on each edge of the graph, generating 0 or more messages to be + * sent to either vertex in the edge. The `mergeMsg` function is then used to combine all + * messages destined to the same vertex. + * + * @tparam A + * the type of message to be sent to each vertex + * + * @param sendMsg + * runs on each edge, sending messages to neighboring vertices using the [[EdgeContext]]. + * @param mergeMsg + * used to combine messages from `sendMsg` destined to the same vertex. This combiner should + * be commutative and associative. + * @param tripletFields + * which fields should be included in the [[EdgeContext]] passed to the `sendMsg` function. If + * not all fields are needed, specifying this can improve performance. + * + * @example + * We can use this function to compute the in-degree of each vertex + * {{{ + * val rawGraph: Graph[_, _] = Graph.textFile("twittergraph") + * val inDeg: RDD[(VertexId, Int)] = + * rawGraph.aggregateMessages[Int](ctx => ctx.sendToDst(1), _ + _) + * }}} + * + * @note + * By expressing computation at the edge level we achieve maximum parallelism. This is one of + * the core functions in the Graph API that enables neighborhood level computation. For + * example this function can be used to count neighbors satisfying a predicate or implement + * PageRank. + */ + def aggregateMessages[A: ClassTag]( + sendMsg: EdgeContext[VD, ED, A] => Unit, + mergeMsg: (A, A) => A, + tripletFields: TripletFields = TripletFields.All): VertexRDD[A] = { + aggregateMessagesWithActiveSet(sendMsg, mergeMsg, tripletFields, None) + } + + /** + * Aggregates values from the neighboring edges and vertices of each vertex. The user-supplied + * `sendMsg` function is invoked on each edge of the graph, generating 0 or more messages to be + * sent to either vertex in the edge. The `mergeMsg` function is then used to combine all + * messages destined to the same vertex. + * + * This variant can take an active set to restrict the computation and is intended for internal + * use only. + * + * @tparam A + * the type of message to be sent to each vertex + * + * @param sendMsg + * runs on each edge, sending messages to neighboring vertices using the [[EdgeContext]]. + * @param mergeMsg + * used to combine messages from `sendMsg` destined to the same vertex. This combiner should + * be commutative and associative. + * @param tripletFields + * which fields should be included in the [[EdgeContext]] passed to the `sendMsg` function. If + * not all fields are needed, specifying this can improve performance. + * @param activeSetOpt + * an efficient way to run the aggregation on a subset of the edges if desired. This is done + * by specifying a set of "active" vertices and an edge direction. The `sendMsg` function will + * then run on only edges connected to active vertices by edges in the specified direction. If + * the direction is `In`, `sendMsg` will only be run on edges with destination in the active + * set. If the direction is `Out`, `sendMsg` will only be run on edges originating from + * vertices in the active set. If the direction is `Either`, `sendMsg` will be run on edges + * with *either* vertex in the active set. If the direction is `Both`, `sendMsg` will be run + * on edges with *both* vertices in the active set. The active set must have the same index as + * the graph's vertices. + */ + private[graphx] def aggregateMessagesWithActiveSet[A: ClassTag]( + sendMsg: EdgeContext[VD, ED, A] => Unit, + mergeMsg: (A, A) => A, + tripletFields: TripletFields, + activeSetOpt: Option[(VertexRDD[_], EdgeDirection)]): VertexRDD[A] + + /** + * Joins the vertices with entries in the `table` RDD and merges the results using `mapFunc`. + * The input table should contain at most one entry for each vertex. If no entry in `other` is + * provided for a particular vertex in the graph, the map function receives `None`. + * + * @tparam U + * the type of entry in the table of updates + * @tparam VD2 + * the new vertex value type + * + * @param other + * the table to join with the vertices in the graph. The table should contain at most one + * entry for each vertex. + * @param mapFunc + * the function used to compute the new vertex values. The map function is invoked for all + * vertices, even those that do not have a corresponding entry in the table. + * + * @example + * This function is used to update the vertices with new values based on external data. For + * example we could add the out-degree to each vertex record: + * + * {{{ + * val rawGraph: Graph[_, _] = Graph.textFile("webgraph") + * val outDeg: RDD[(VertexId, Int)] = rawGraph.outDegrees + * val graph = rawGraph.outerJoinVertices(outDeg) { + * (vid, data, optDeg) => optDeg.getOrElse(0) + * } + * }}} + */ + def outerJoinVertices[U: ClassTag, VD2: ClassTag](other: RDD[(VertexId, U)])( + mapFunc: (VertexId, VD, Option[U]) => VD2)(implicit eq: VD =:= VD2 = null): Graph[VD2, ED] + + /** + * The associated [[GraphOps]] object. + */ + // Save a copy of the GraphOps object so there is always one unique GraphOps object + // for a given Graph object, and thus the lazy vals in GraphOps would work as intended. + val ops = new GraphOps(this) +} // end of Graph + +/** + * The Graph object contains a collection of routines used to construct graphs from RDDs. + */ +object Graph { + + /** + * Construct a graph from a collection of edges encoded as vertex id pairs. + * + * @param rawEdges + * a collection of edges in (src, dst) form + * @param defaultValue + * the vertex attributes with which to create vertices referenced by the edges + * @param uniqueEdges + * if multiple identical edges are found they are combined and the edge attribute is set to + * the sum. Otherwise duplicate edges are treated as separate. To enable `uniqueEdges`, a + * [[PartitionStrategy]] must be provided. + * @param edgeStorageLevel + * the desired storage level at which to cache the edges if necessary + * @param vertexStorageLevel + * the desired storage level at which to cache the vertices if necessary + * + * @return + * a graph with edge attributes containing either the count of duplicate edges or 1 (if + * `uniqueEdges` is `None`) and vertex attributes containing the total degree of each vertex. + */ + def fromEdgeTuples[VD: ClassTag]( + rawEdges: RDD[(VertexId, VertexId)], + defaultValue: VD, + uniqueEdges: Option[PartitionStrategy] = None, + edgeStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY, + vertexStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, Int] = { + val edges = rawEdges.map(p => Edge(p._1, p._2, 1)) + val graph = GraphImpl(edges, defaultValue, edgeStorageLevel, vertexStorageLevel) + uniqueEdges match { + case Some(p) => graph.partitionBy(p).groupEdges((a, b) => a + b) + case None => graph + } + } + + /** + * Construct a graph from a collection of edges. + * + * @param edges + * the RDD containing the set of edges in the graph + * @param defaultValue + * the default vertex attribute to use for each vertex + * @param edgeStorageLevel + * the desired storage level at which to cache the edges if necessary + * @param vertexStorageLevel + * the desired storage level at which to cache the vertices if necessary + * + * @return + * a graph with edge attributes described by `edges` and vertices given by all vertices in + * `edges` with value `defaultValue` + */ + def fromEdges[VD: ClassTag, ED: ClassTag]( + edges: RDD[Edge[ED]], + defaultValue: VD, + edgeStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY, + vertexStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED] = { + GraphImpl[VD, ED](edges, defaultValue, edgeStorageLevel, vertexStorageLevel) + } + + /** + * Construct a graph from a collection of vertices and edges with attributes. Duplicate vertices + * are picked arbitrarily and vertices found in the edge collection but not in the input + * vertices are assigned the default attribute. + * + * @tparam VD + * the vertex attribute type + * @tparam ED + * the edge attribute type + * @param vertices + * the "set" of vertices and their attributes + * @param edges + * the collection of edges in the graph + * @param defaultVertexAttr + * the default vertex attribute to use for vertices that are mentioned in edges but not in + * vertices + * @param edgeStorageLevel + * the desired storage level at which to cache the edges if necessary + * @param vertexStorageLevel + * the desired storage level at which to cache the vertices if necessary + */ + def apply[VD: ClassTag, ED: ClassTag]( + vertices: RDD[(VertexId, VD)], + edges: RDD[Edge[ED]], + defaultVertexAttr: VD = null.asInstanceOf[VD], + edgeStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY, + vertexStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED] = { + GraphImpl[VD, ED](vertices, edges, defaultVertexAttr, edgeStorageLevel, vertexStorageLevel) + } + + /** + * Implicitly extracts the [[GraphOps]] member from a graph. + * + * To improve modularity the Graph type only contains a small set of basic operations. All the + * convenience operations are defined in the [[GraphOps]] class which may be shared across + * multiple graph implementations. + */ + @nowarn + implicit def graphToGraphOps[VD: ClassTag, ED: ClassTag](g: Graph[VD, ED]): GraphOps[VD, ED] = + g.ops +} // end of Graph object diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/GraphLoader.scala new file mode 100644 index 000000000..272a864fc --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/GraphLoader.scala @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.apache.spark.SparkContext +import org.apache.spark.graphframes.graphx.impl.EdgePartitionBuilder +import org.apache.spark.graphframes.graphx.impl.GraphImpl +import org.apache.spark.internal.Logging +import org.apache.spark.storage.StorageLevel + +/** + * Provides utilities for loading [[Graph]]s from files. + */ +object GraphLoader extends Logging { + + /** + * Loads a graph from an edge list formatted file where each line contains two integers: a + * source id and a target id. Skips lines that begin with `#`. + * + * If desired the edges can be automatically oriented in the positive direction (source Id is + * less than target Id) by setting `canonicalOrientation` to true. + * + * @example + * Loads a file in the following format: + * {{{ + * # Comment Line + * # Source Id <\t> Target Id + * 1 -5 + * 1 2 + * 2 7 + * 1 8 + * }}} + * + * @param sc + * SparkContext + * @param path + * the path to the file (e.g., /home/data/file or hdfs://file) + * @param canonicalOrientation + * whether to orient edges in the positive direction + * @param numEdgePartitions + * the number of partitions for the edge RDD Setting this value to -1 will use the default + * parallelism. + * @param edgeStorageLevel + * the desired storage level for the edge partitions + * @param vertexStorageLevel + * the desired storage level for the vertex partitions + */ + def edgeListFile( + sc: SparkContext, + path: String, + canonicalOrientation: Boolean = false, + numEdgePartitions: Int = -1, + edgeStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY, + vertexStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[Int, Int] = { + + // Parse the edge data table directly into edge partitions + val lines = + if (numEdgePartitions > 0) { + sc.textFile(path, numEdgePartitions).coalesce(numEdgePartitions) + } else { + sc.textFile(path) + } + val edges = lines + .mapPartitionsWithIndex { (pid, iter) => + val builder = new EdgePartitionBuilder[Int, Int] + iter.foreach { line => + if (!line.isEmpty && line(0) != '#') { + val lineArray = line.split("\\s+") + if (lineArray.length < 2) { + throw new IllegalArgumentException("Invalid line: " + line) + } + val srcId = lineArray(0).toLong + val dstId = lineArray(1).toLong + if (canonicalOrientation && srcId > dstId) { + builder.add(dstId, srcId, 1) + } else { + builder.add(srcId, dstId, 1) + } + } + } + Iterator((pid, builder.toEdgePartition)) + } + .persist(edgeStorageLevel) + .setName("GraphLoader.edgeListFile - edges (%s)".format(path)) + edges.count() + + GraphImpl.fromEdgePartitions( + edges, + defaultVertexAttr = 1, + edgeStorageLevel = edgeStorageLevel, + vertexStorageLevel = vertexStorageLevel) + } // end of edgeListFile + +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/GraphOps.scala new file mode 100644 index 000000000..93c68255a --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/GraphOps.scala @@ -0,0 +1,522 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.apache.spark.SparkException +import org.apache.spark.graphframes.graphx.lib.ConnectedComponents +import org.apache.spark.graphframes.graphx.lib.PageRank +import org.apache.spark.graphframes.graphx.lib.StronglyConnectedComponents +import org.apache.spark.graphframes.graphx.lib.TriangleCount +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.rdd.RDD + +import scala.reflect.ClassTag +import scala.util.Random + +/** + * Contains additional functionality for [[Graph]]. All operations are expressed in terms of the + * efficient GraphX API. This class is implicitly constructed for each Graph object. + * + * @tparam VD + * the vertex attribute type + * @tparam ED + * the edge attribute type + */ +class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Serializable { + + /** The number of edges in the graph. */ + @transient lazy val numEdges: Long = graph.edges.count() + + /** The number of vertices in the graph. */ + @transient lazy val numVertices: Long = graph.vertices.count() + + /** + * The in-degree of each vertex in the graph. + * @note + * Vertices with no in-edges are not returned in the resulting RDD. + */ + @transient lazy val inDegrees: VertexRDD[Int] = + degreesRDD(EdgeDirection.In).setName("GraphOps.inDegrees") + + /** + * The out-degree of each vertex in the graph. + * @note + * Vertices with no out-edges are not returned in the resulting RDD. + */ + @transient lazy val outDegrees: VertexRDD[Int] = + degreesRDD(EdgeDirection.Out).setName("GraphOps.outDegrees") + + /** + * The degree of each vertex in the graph. + * @note + * Vertices with no edges are not returned in the resulting RDD. + */ + @transient lazy val degrees: VertexRDD[Int] = + degreesRDD(EdgeDirection.Either).setName("GraphOps.degrees") + + /** + * Computes the neighboring vertex degrees. + * + * @param edgeDirection + * the direction along which to collect neighboring vertex attributes + */ + private def degreesRDD(edgeDirection: EdgeDirection): VertexRDD[Int] = { + if (edgeDirection == EdgeDirection.In) { + graph.aggregateMessages(_.sendToDst(1), _ + _, TripletFields.None) + } else if (edgeDirection == EdgeDirection.Out) { + graph.aggregateMessages(_.sendToSrc(1), _ + _, TripletFields.None) + } else { // EdgeDirection.Either + graph.aggregateMessages( + ctx => { ctx.sendToSrc(1); ctx.sendToDst(1) }, + _ + _, + TripletFields.None) + } + } + + /** + * Collect the neighbor vertex ids for each vertex. + * + * @param edgeDirection + * the direction along which to collect neighboring vertices + * + * @return + * the set of neighboring ids for each vertex + */ + def collectNeighborIds(edgeDirection: EdgeDirection): VertexRDD[Array[VertexId]] = { + val nbrs = + if (edgeDirection == EdgeDirection.Either) { + graph.aggregateMessages[Array[VertexId]]( + ctx => { ctx.sendToSrc(Array(ctx.dstId)); ctx.sendToDst(Array(ctx.srcId)) }, + _ ++ _, + TripletFields.None) + } else if (edgeDirection == EdgeDirection.Out) { + graph.aggregateMessages[Array[VertexId]]( + ctx => ctx.sendToSrc(Array(ctx.dstId)), + _ ++ _, + TripletFields.None) + } else if (edgeDirection == EdgeDirection.In) { + graph.aggregateMessages[Array[VertexId]]( + ctx => ctx.sendToDst(Array(ctx.srcId)), + _ ++ _, + TripletFields.None) + } else { + throw new SparkException( + "It doesn't make sense to collect neighbor ids without a " + + "direction. (EdgeDirection.Both is not supported; use EdgeDirection.Either instead.)") + } + graph.vertices.leftZipJoin(nbrs) { (_, _, nbrsOpt) => + nbrsOpt.getOrElse(Array.empty[VertexId]) + } + } // end of collectNeighborIds + + /** + * Collect the neighbor vertex attributes for each vertex. + * + * @note + * This function could be highly inefficient on power-law graphs where high degree vertices + * may force a large amount of information to be collected to a single location. + * + * @param edgeDirection + * the direction along which to collect neighboring vertices + * + * @return + * the vertex set of neighboring vertex attributes for each vertex + */ + def collectNeighbors(edgeDirection: EdgeDirection): VertexRDD[Array[(VertexId, VD)]] = { + val nbrs = edgeDirection match { + case EdgeDirection.Either => + graph.aggregateMessages[Array[(VertexId, VD)]]( + ctx => { + ctx.sendToSrc(Array((ctx.dstId, ctx.dstAttr))) + ctx.sendToDst(Array((ctx.srcId, ctx.srcAttr))) + }, + (a, b) => a ++ b, + TripletFields.All) + case EdgeDirection.In => + graph.aggregateMessages[Array[(VertexId, VD)]]( + ctx => ctx.sendToDst(Array((ctx.srcId, ctx.srcAttr))), + (a, b) => a ++ b, + TripletFields.Src) + case EdgeDirection.Out => + graph.aggregateMessages[Array[(VertexId, VD)]]( + ctx => ctx.sendToSrc(Array((ctx.dstId, ctx.dstAttr))), + (a, b) => a ++ b, + TripletFields.Dst) + case EdgeDirection.Both => + throw new SparkException( + "collectEdges does not support EdgeDirection.Both. Use" + + "EdgeDirection.Either instead.") + // unreachable + case _ => throw new SparkException("Unknown edge direction") + } + graph.vertices.leftJoin(nbrs) { (_, _, nbrsOpt) => + nbrsOpt.getOrElse(Array.empty[(VertexId, VD)]) + } + } // end of collectNeighbor + + /** + * Returns an RDD that contains for each vertex v its local edges, i.e., the edges that are + * incident on v, in the user-specified direction. Warning: note that singleton vertices, those + * with no edges in the given direction will not be part of the return value. + * + * @note + * This function could be highly inefficient on power-law graphs where high degree vertices + * may force a large amount of information to be collected to a single location. + * + * @param edgeDirection + * the direction along which to collect the local edges of vertices + * + * @return + * the local edges for each vertex + */ + def collectEdges(edgeDirection: EdgeDirection): VertexRDD[Array[Edge[ED]]] = { + edgeDirection match { + case EdgeDirection.Either => + graph.aggregateMessages[Array[Edge[ED]]]( + ctx => { + ctx.sendToSrc(Array(new Edge(ctx.srcId, ctx.dstId, ctx.attr))) + ctx.sendToDst(Array(new Edge(ctx.srcId, ctx.dstId, ctx.attr))) + }, + (a, b) => a ++ b, + TripletFields.EdgeOnly) + case EdgeDirection.In => + graph.aggregateMessages[Array[Edge[ED]]]( + ctx => ctx.sendToDst(Array(new Edge(ctx.srcId, ctx.dstId, ctx.attr))), + (a, b) => a ++ b, + TripletFields.EdgeOnly) + case EdgeDirection.Out => + graph.aggregateMessages[Array[Edge[ED]]]( + ctx => ctx.sendToSrc(Array(new Edge(ctx.srcId, ctx.dstId, ctx.attr))), + (a, b) => a ++ b, + TripletFields.EdgeOnly) + case EdgeDirection.Both => + throw new SparkException( + "collectEdges does not support EdgeDirection.Both. Use" + + "EdgeDirection.Either instead.") + // unreachable + case _ => throw new SparkException("Unknown edge direction") + } + } + + /** + * Remove self edges. + * + * @return + * a graph with all self edges removed + */ + def removeSelfEdges(): Graph[VD, ED] = { + graph.subgraph(epred = e => e.srcId != e.dstId) + } + + /** + * Join the vertices with an RDD and then apply a function from the vertex and RDD entry to a + * new vertex value. The input table should contain at most one entry for each vertex. If no + * entry is provided the map function is skipped and the old value is used. + * + * @tparam U + * the type of entry in the table of updates + * @param table + * the table to join with the vertices in the graph. The table should contain at most one + * entry for each vertex. + * @param mapFunc + * the function used to compute the new vertex values. The map function is invoked only for + * vertices with a corresponding entry in the table otherwise the old vertex value is used. + * + * @example + * This function is used to update the vertices with new values based on external data. For + * example we could add the out degree to each vertex record + * + * {{{ + * val rawGraph: Graph[Int, Int] = GraphLoader.edgeListFile(sc, "webgraph") + * .mapVertices((_, _) => 0) + * val outDeg = rawGraph.outDegrees + * val graph = rawGraph.joinVertices[Int](outDeg) + * ((_, _, outDeg) => outDeg) + * }}} + */ + def joinVertices[U: ClassTag](table: RDD[(VertexId, U)])( + mapFunc: (VertexId, VD, U) => VD): Graph[VD, ED] = { + val uf = (id: VertexId, data: VD, o: Option[U]) => { + o match { + case Some(u) => mapFunc(id, data, u) + case None => data + } + } + graph.outerJoinVertices(table)(uf) + } + + /** + * Filter the graph by computing some values to filter on, and applying the predicates. + * + * @param preprocess + * a function to compute new vertex and edge data before filtering + * @param epred + * edge pred to filter on after preprocess, see more details under + * [[org.apache.spark.graphframes.graphx.Graph#subgraph]] + * @param vpred + * vertex pred to filter on after preprocess, see more details under + * [[org.apache.spark.graphframes.graphx.Graph#subgraph]] + * @tparam VD2 + * vertex type the vpred operates on + * @tparam ED2 + * edge type the epred operates on + * @return + * a subgraph of the original graph, with its data unchanged + * @example + * This function can be used to filter the graph based on some property, without changing the + * vertex and edge values in your program. For example, we could remove the vertices in a + * graph with 0 outdegree + * + * {{{ + * graph.filter( + * graph => { + * val degrees: VertexRDD[Int] = graph.outDegrees + * graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)} + * }, + * vpred = (vid: VertexId, deg:Int) => deg > 0 + * ) + * }}} + */ + def filter[VD2: ClassTag, ED2: ClassTag]( + preprocess: Graph[VD, ED] => Graph[VD2, ED2], + epred: (EdgeTriplet[VD2, ED2]) => Boolean = (_: EdgeTriplet[VD2, ED2]) => true, + vpred: (VertexId, VD2) => Boolean = (_: VertexId, _: VD2) => true): Graph[VD, ED] = { + graph.mask(preprocess(graph).subgraph(epred, vpred)) + } + + /** + * Picks a random vertex from the graph and returns its ID. + */ + def pickRandomVertex(): VertexId = { + val probability = 50.0 / graph.numVertices + var found = false + var retVal: VertexId = null.asInstanceOf[VertexId] + while (!found) { + val selectedVertices = graph.vertices.flatMap { vidVvals => + if (Random.nextDouble() < probability) { Some(vidVvals._1) } + else { None } + } + if (selectedVertices.count() > 0) { + found = true + val collectedVertices = selectedVertices.collect() + retVal = collectedVertices(Random.nextInt(collectedVertices.length)) + } + } + retVal + } + + /** + * Convert bi-directional edges into uni-directional ones. Some graph algorithms (e.g., + * TriangleCount) assume that an input graph has its edges in canonical direction. This function + * rewrites the vertex ids of edges so that srcIds are smaller than dstIds, and merges the + * duplicated edges. + * + * @param mergeFunc + * the user defined reduce function which should be commutative and associative and is used to + * combine the output of the map phase + * + * @return + * the resulting graph with canonical edges + */ + def convertToCanonicalEdges(mergeFunc: (ED, ED) => ED = (e1, _) => e1): Graph[VD, ED] = { + val newEdges = + graph.edges + .map { + case e if e.srcId < e.dstId => ((e.srcId, e.dstId), e.attr) + case e => ((e.dstId, e.srcId), e.attr) + } + .reduceByKey(mergeFunc) + .map(e => new Edge(e._1._1, e._1._2, e._2)) + Graph[VD, ED](graph.vertices, newEdges) + } + + /** + * Execute a Pregel-like iterative vertex-parallel abstraction. The user-defined vertex-program + * `vprog` is executed in parallel on each vertex receiving any inbound messages and computing a + * new value for the vertex. The `sendMsg` function is then invoked on all out-edges and is used + * to compute an optional message to the destination vertex. The `mergeMsg` function is a + * commutative associative function used to combine messages destined to the same vertex. + * + * On the first iteration all vertices receive the `initialMsg` and on subsequent iterations if + * a vertex does not receive a message then the vertex-program is not invoked. + * + * This function iterates until there are no remaining messages, or for `maxIterations` + * iterations. + * + * @tparam A + * the Pregel message type + * + * @param initialMsg + * the message each vertex will receive at the on the first iteration + * + * @param maxIterations + * the maximum number of iterations to run for + * + * @param activeDirection + * the direction of edges incident to a vertex that received a message in the previous round + * on which to run `sendMsg`. For example, if this is `EdgeDirection.Out`, only out-edges of + * vertices that received a message in the previous round will run. + * + * @param vprog + * the user-defined vertex program which runs on each vertex and receives the inbound message + * and computes a new vertex value. On the first iteration the vertex program is invoked on + * all vertices and is passed the default message. On subsequent iterations the vertex program + * is only invoked on those vertices that receive messages. + * + * @param sendMsg + * a user supplied function that is applied to out edges of vertices that received messages in + * the current iteration + * + * @param mergeMsg + * a user supplied function that takes two incoming messages of type A and merges them into a + * single message of type A. ''This function must be commutative and associative and ideally + * the size of A should not increase.'' + * + * @return + * the resulting graph at the end of the computation + */ + def pregel[A: ClassTag]( + initialMsg: A, + maxIterations: Int = Int.MaxValue, + activeDirection: EdgeDirection = EdgeDirection.Either)( + vprog: (VertexId, VD, A) => VD, + sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)], + mergeMsg: (A, A) => A): Graph[VD, ED] = { + Pregel(graph, initialMsg, maxIterations, activeDirection)(vprog, sendMsg, mergeMsg) + } + + /** + * Run a dynamic version of PageRank returning a graph with vertex attributes containing the + * PageRank and edge attributes containing the normalized edge weight. + * + * @see + * [[org.apache.spark.graphframes.graphx.lib.PageRank$#runUntilConvergence]] + */ + def pageRank(tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = { + PageRank.runUntilConvergence(graph, tol, resetProb) + } + + /** + * Run personalized PageRank for a given vertex, such that all random walks are started relative + * to the source node. + * + * @see + * [[org.apache.spark.graphframes.graphx.lib.PageRank$#runUntilConvergenceWithOptions]] + */ + def personalizedPageRank( + src: VertexId, + tol: Double, + resetProb: Double = 0.15): Graph[Double, Double] = { + PageRank.runUntilConvergenceWithOptions(graph, tol, resetProb, Some(src)) + } + + /** + * Run parallel personalized PageRank for a given array of source vertices, such that all random + * walks are started relative to the source vertices + */ + def staticParallelPersonalizedPageRank( + sources: Array[VertexId], + numIter: Int, + resetProb: Double = 0.15): Graph[Vector, Double] = { + PageRank.runParallelPersonalizedPageRank(graph, numIter, resetProb, sources) + } + + /** + * Run Personalized PageRank for a fixed number of iterations with with all iterations + * originating at the source node returning a graph with vertex attributes containing the + * PageRank and edge attributes the normalized edge weight. + * + * @see + * [[org.apache.spark.graphframes.graphx.lib.PageRank$#runWithOptions]] + */ + def staticPersonalizedPageRank( + src: VertexId, + numIter: Int, + resetProb: Double = 0.15): Graph[Double, Double] = { + PageRank.runWithOptions(graph, numIter, resetProb, Some(src)) + } + + /** + * Run PageRank for a fixed number of iterations returning a graph with vertex attributes + * containing the PageRank and edge attributes the normalized edge weight. + * + * @see + * [[org.apache.spark.graphframes.graphx.lib.PageRank$#run]] + */ + def staticPageRank(numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = { + PageRank.run(graph, numIter, resetProb) + } + + /** + * Run PageRank for a fixed number of iterations returning a graph with vertex attributes + * containing the PageRank and edge attributes the normalized edge weight, optionally including + * including a previous pageRank computation to be used as a start point for the new iterations + * + * @see + * [[org.apache.spark.graphframes.graphx.lib.PageRank$#runWithOptionsWithPreviousPageRank]] + */ + def staticPageRank( + numIter: Int, + resetProb: Double, + prePageRank: Graph[Double, Double]): Graph[Double, Double] = { + PageRank.runWithOptionsWithPreviousPageRank(graph, numIter, resetProb, None, prePageRank) + } + + /** + * Compute the connected component membership of each vertex and return a graph with the vertex + * value containing the lowest vertex id in the connected component containing that vertex. + * + * @see + * `org.apache.spark.graphx.lib.ConnectedComponents.run` + */ + def connectedComponents(): Graph[VertexId, ED] = { + ConnectedComponents.run(graph) + } + + /** + * Compute the connected component membership of each vertex and return a graph with the vertex + * value containing the lowest vertex id in the connected component containing that vertex. + * + * @see + * `org.apache.spark.graphx.lib.ConnectedComponents.run` + */ + def connectedComponents(maxIterations: Int): Graph[VertexId, ED] = { + ConnectedComponents.run(graph, maxIterations) + } + + /** + * Compute the number of triangles passing through each vertex. + * + * @see + * [[org.apache.spark.graphframes.graphx.lib.TriangleCount$#run]] + */ + def triangleCount(): Graph[Int, ED] = { + TriangleCount.run(graph) + } + + /** + * Compute the strongly connected component (SCC) of each vertex and return a graph with the + * vertex value containing the lowest vertex id in the SCC containing that vertex. + * + * @see + * [[org.apache.spark.graphframes.graphx.lib.StronglyConnectedComponents$#run]] + */ + def stronglyConnectedComponents(numIter: Int): Graph[VertexId, ED] = { + StronglyConnectedComponents.run(graph, numIter) + } +} // end of GraphOps diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/GraphXUtils.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/GraphXUtils.scala new file mode 100644 index 000000000..8d2118800 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/GraphXUtils.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.apache.spark.SparkConf +import org.apache.spark.graphframes.graphx.impl.EdgePartition +import org.apache.spark.graphframes.graphx.impl.RoutingTablePartition +import org.apache.spark.graphframes.graphx.impl.ShippableVertexPartition +import org.apache.spark.graphframes.graphx.impl.VertexAttributeBlock +import org.apache.spark.graphframes.graphx.impl.VertexIdToIndexMap +import org.apache.spark.graphframes.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.BoundedPriorityQueue +import org.apache.spark.util.collection.BitSet +import org.apache.spark.util.collection.OpenHashSet + +import scala.annotation.nowarn +import scala.reflect.ClassTag + +object GraphXUtils { + + /** + * Registers classes that GraphX uses with Kryo. + */ + def registerKryoClasses(conf: SparkConf): Unit = { + val _ = conf.registerKryoClasses( + Array( + classOf[Edge[Object]], + classOf[(VertexId, Object)], + classOf[EdgePartition[Object, Object]], + classOf[ShippableVertexPartition[Object]], + classOf[RoutingTablePartition], + classOf[BitSet], + classOf[VertexIdToIndexMap], + classOf[VertexAttributeBlock[Object]], + classOf[PartitionStrategy], + classOf[BoundedPriorityQueue[Object]], + classOf[EdgeDirection], + classOf[GraphXPrimitiveKeyOpenHashMap[VertexId, Int]], + classOf[OpenHashSet[Int]], + classOf[OpenHashSet[Long]])) + () + } + + /** + * A proxy method to map the obsolete API to the new one. + */ + @nowarn + private[graphx] def mapReduceTriplets[VD: ClassTag, ED: ClassTag, A: ClassTag]( + g: Graph[VD, ED], + mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)], + reduceFunc: (A, A) => A, + activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None): VertexRDD[A] = { + def sendMsg(ctx: EdgeContext[VD, ED, A]): Unit = { + mapFunc(ctx.toEdgeTriplet).foreach { kv => + val id = kv._1 + val msg = kv._2 + if (id == ctx.srcId) { + ctx.sendToSrc(msg) + } else { + assert(id == ctx.dstId) + ctx.sendToDst(msg) + } + } + } + g.aggregateMessagesWithActiveSet(sendMsg, reduceFunc, TripletFields.All, activeSetOpt) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/PartitionStrategy.scala new file mode 100644 index 000000000..fc44473f2 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/PartitionStrategy.scala @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +/** + * Represents the way edges are assigned to edge partitions based on their source and destination + * vertex IDs. + */ +trait PartitionStrategy extends Serializable { + + /** Returns the partition number for a given edge. */ + def getPartition(src: VertexId, dst: VertexId, numParts: PartitionID): PartitionID +} + +/** + * Collection of built-in [[PartitionStrategy]] implementations. + */ +object PartitionStrategy { + + /** + * Assigns edges to partitions using a 2D partitioning of the sparse edge adjacency matrix, + * guaranteeing a `2 * sqrt(numParts)` bound on vertex replication. + * + * Suppose we have a graph with 12 vertices that we want to partition over 9 machines. We can + * use the following sparse matrix representation: + * + *
__________________________________ v0 | P0 * | P1 | P2 * | v1 | **** | * | | v2 | + * ******* | ** | **** | v3 | ***** | * * | * | ---------------------------------- v4 | P3 * | + * P4 *** | P5 ** * | v5 | * * | * | | v6 | * | ** | **** | v7 | * * * | * * | * | + * ---------------------------------- v8 | P6 * | P7 * | P8 * *| v9 | * | * * | | v10 | * | ** | + * * * | v11 | * <-E | *** | ** | ----------------------------------+ * + * The edge denoted by `E` connects `v11` with `v1` and is assigned to processor `P6`. To get + * the processor number we divide the matrix into `sqrt(numParts)` by `sqrt(numParts)` blocks. + * Notice that edges adjacent to `v11` can only be in the first column of blocks `(P0, P3, P6)` + * or the last row of blocks `(P6, P7, P8)`. As a consequence we can guarantee that `v11` will + * need to be replicated to at most `2 * sqrt(numParts)` machines. + * + * Notice that `P0` has many edges and as a consequence this partitioning would lead to poor + * work balance. To improve balance we first multiply each vertex id by a large prime to shuffle + * the vertex locations. + * + * When the number of partitions requested is not a perfect square we use a slightly different + * method where the last column can have a different number of rows than the others while still + * maintaining the same size per block. + */ + case object EdgePartition2D extends PartitionStrategy { + override def getPartition( + src: VertexId, + dst: VertexId, + numParts: PartitionID): PartitionID = { + val ceilSqrtNumParts: PartitionID = math.ceil(math.sqrt(numParts.toDouble)).toInt + val mixingPrime: VertexId = 1125899906842597L + if (numParts == ceilSqrtNumParts * ceilSqrtNumParts) { + // Use old method for perfect squared to ensure we get same results + val col: PartitionID = (math.abs(src * mixingPrime) % ceilSqrtNumParts).toInt + val row: PartitionID = (math.abs(dst * mixingPrime) % ceilSqrtNumParts).toInt + (col * ceilSqrtNumParts + row) % numParts + + } else { + // Otherwise use new method + val cols = ceilSqrtNumParts + val rows = (numParts + cols - 1) / cols + val lastColRows = numParts - rows * (cols - 1) + val col = (math.abs(src * mixingPrime) % numParts / rows).toInt + val row = + (math.abs(dst * mixingPrime) % (if (col < cols - 1) rows else lastColRows)).toInt + col * rows + row + + } + } + } + + /** + * Assigns edges to partitions using only the source vertex ID, colocating edges with the same + * source. + */ + case object EdgePartition1D extends PartitionStrategy { + override def getPartition( + src: VertexId, + dst: VertexId, + numParts: PartitionID): PartitionID = { + val mixingPrime: VertexId = 1125899906842597L + (math.abs(src * mixingPrime) % numParts).toInt + } + } + + /** + * Assigns edges to partitions by hashing the source and destination vertex IDs, resulting in a + * random vertex cut that colocates all same-direction edges between two vertices. + */ + case object RandomVertexCut extends PartitionStrategy { + override def getPartition( + src: VertexId, + dst: VertexId, + numParts: PartitionID): PartitionID = { + math.abs((src, dst).hashCode()) % numParts + } + } + + /** + * Assigns edges to partitions by hashing the source and destination vertex IDs in a canonical + * direction, resulting in a random vertex cut that colocates all edges between two vertices, + * regardless of direction. + */ + case object CanonicalRandomVertexCut extends PartitionStrategy { + override def getPartition( + src: VertexId, + dst: VertexId, + numParts: PartitionID): PartitionID = { + if (src < dst) { + math.abs((src, dst).hashCode()) % numParts + } else { + math.abs((dst, src).hashCode()) % numParts + } + } + } + + /** Returns the PartitionStrategy with the specified name. */ + def fromString(s: String): PartitionStrategy = s match { + case "RandomVertexCut" => RandomVertexCut + case "EdgePartition1D" => EdgePartition1D + case "EdgePartition2D" => EdgePartition2D + case "CanonicalRandomVertexCut" => CanonicalRandomVertexCut + case _ => throw new IllegalArgumentException("Invalid PartitionStrategy: " + s) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/Pregel.scala new file mode 100644 index 000000000..d28e55370 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/Pregel.scala @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.apache.spark.graphframes.graphx.util.PeriodicGraphCheckpointer +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.util.PeriodicRDDCheckpointer + +import scala.reflect.ClassTag + +/** + * Implements a Pregel-like bulk-synchronous message-passing API. + * + * Unlike the original Pregel API, the GraphX Pregel API factors the sendMessage computation over + * edges, enables the message sending computation to read both vertex attributes, and constrains + * messages to the graph structure. These changes allow for substantially more efficient + * distributed execution while also exposing greater flexibility for graph-based computation. + * + * @example + * We can use the Pregel abstraction to implement PageRank: + * {{{ + * val pagerankGraph: Graph[Double, Double] = graph + * // Associate the degree with each vertex + * .outerJoinVertices(graph.outDegrees) { + * (vid, vdata, deg) => deg.getOrElse(0) + * } + * // Set the weight on the edges based on the degree + * .mapTriplets(e => 1.0 / e.srcAttr) + * // Set the vertex attributes to the initial pagerank values + * .mapVertices((id, attr) => 1.0) + * + * def vertexProgram(id: VertexId, attr: Double, msgSum: Double): Double = + * resetProb + (1.0 - resetProb) * msgSum + * def sendMessage(id: VertexId, edge: EdgeTriplet[Double, Double]): Iterator[(VertexId, Double)] = + * Iterator((edge.dstId, edge.srcAttr * edge.attr)) + * def messageCombiner(a: Double, b: Double): Double = a + b + * val initialMessage = 0.0 + * // Execute Pregel for a fixed number of iterations. + * Pregel(pagerankGraph, initialMessage, numIter)( + * vertexProgram, sendMessage, messageCombiner) + * }}} + */ +object Pregel extends Logging { + + /** + * Execute a Pregel-like iterative vertex-parallel abstraction. The user-defined vertex-program + * `vprog` is executed in parallel on each vertex receiving any inbound messages and computing a + * new value for the vertex. The `sendMsg` function is then invoked on all out-edges and is used + * to compute an optional message to the destination vertex. The `mergeMsg` function is a + * commutative associative function used to combine messages destined to the same vertex. + * + * On the first iteration all vertices receive the `initialMsg` and on subsequent iterations if + * a vertex does not receive a message then the vertex-program is not invoked. + * + * This function iterates until there are no remaining messages, or for `maxIterations` + * iterations. + * + * @tparam VD + * the vertex data type + * @tparam ED + * the edge data type + * @tparam A + * the Pregel message type + * + * @param graph + * the input graph. + * + * @param initialMsg + * the message each vertex will receive at the first iteration + * + * @param maxIterations + * the maximum number of iterations to run for + * + * @param activeDirection + * the direction of edges incident to a vertex that received a message in the previous round + * on which to run `sendMsg`. For example, if this is `EdgeDirection.Out`, only out-edges of + * vertices that received a message in the previous round will run. The default is + * `EdgeDirection.Either`, which will run `sendMsg` on edges where either side received a + * message in the previous round. If this is `EdgeDirection.Both`, `sendMsg` will only run on + * edges where *both* vertices received a message. + * + * @param vprog + * the user-defined vertex program which runs on each vertex and receives the inbound message + * and computes a new vertex value. On the first iteration the vertex program is invoked on + * all vertices and is passed the default message. On subsequent iterations the vertex program + * is only invoked on those vertices that receive messages. + * + * @param sendMsg + * a user supplied function that is applied to out edges of vertices that received messages in + * the current iteration + * + * @param mergeMsg + * a user supplied function that takes two incoming messages of type A and merges them into a + * single message of type A. ''This function must be commutative and associative and ideally + * the size of A should not increase.'' + * + * @return + * the resulting graph at the end of the computation + */ + def apply[VD: ClassTag, ED: ClassTag, A: ClassTag]( + graph: Graph[VD, ED], + initialMsg: A, + maxIterations: Int = Int.MaxValue, + activeDirection: EdgeDirection = EdgeDirection.Either)( + vprog: (VertexId, VD, A) => VD, + sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)], + mergeMsg: (A, A) => A): Graph[VD, ED] = { + require( + maxIterations > 0, + "Maximum number of iterations must be greater than 0," + + s" but got ${maxIterations}") + + val checkpointInterval = + graph.vertices.sparkContext.getConf.getInt("spark.graphx.pregel.checkpointInterval", -1) + var g = graph.mapVertices((vid, vdata) => vprog(vid, vdata, initialMsg)) + val graphCheckpointer = + new PeriodicGraphCheckpointer[VD, ED](checkpointInterval, graph.vertices.sparkContext) + graphCheckpointer.update(g) + + // compute the messages + var messages = GraphXUtils.mapReduceTriplets(g, sendMsg, mergeMsg) + val messageCheckpointer = + new PeriodicRDDCheckpointer[(VertexId, A)](checkpointInterval, graph.vertices.sparkContext) + messageCheckpointer.update(messages.asInstanceOf[RDD[(VertexId, A)]]) + var isActiveMessagesNonEmpty = !messages.isEmpty() + + // Loop + var prevG: Graph[VD, ED] = null + var i = 0 + while (isActiveMessagesNonEmpty && i < maxIterations) { + // Receive the messages and update the vertices. + prevG = g + g = g.joinVertices(messages)(vprog) + graphCheckpointer.update(g) + + val oldMessages = messages + // Send new messages, skipping edges where neither side received a message. We must cache + // messages so it can be materialized on the next line, allowing us to uncache the previous + // iteration. + messages = + GraphXUtils.mapReduceTriplets(g, sendMsg, mergeMsg, Some((oldMessages, activeDirection))) + // The call to count() materializes `messages` and the vertices of `g`. This hides oldMessages + // (depended on by the vertices of g) and the vertices of prevG (depended on by oldMessages + // and the vertices of g). + messageCheckpointer.update(messages.asInstanceOf[RDD[(VertexId, A)]]) + isActiveMessagesNonEmpty = !messages.isEmpty() + + // Unpersist the RDDs hidden by newly-materialized RDDs + oldMessages.unpersist() + prevG.unpersistVertices() + prevG.edges.unpersist() + // count the iteration + i += 1 + } + messageCheckpointer.unpersistDataSet() + graphCheckpointer.deleteAllCheckpoints() + messageCheckpointer.deleteAllCheckpoints() + g + } // end of apply + +} // end of class Pregel diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/VertexRDD.scala new file mode 100644 index 000000000..6a0ad3481 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/VertexRDD.scala @@ -0,0 +1,417 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.apache.spark._ +import org.apache.spark.graphframes.graphx.impl.RoutingTablePartition +import org.apache.spark.graphframes.graphx.impl.ShippableVertexPartition +import org.apache.spark.graphframes.graphx.impl.VertexAttributeBlock +import org.apache.spark.graphframes.graphx.impl.VertexRDDImpl +import org.apache.spark.rdd._ +import org.apache.spark.storage.StorageLevel + +import scala.reflect.ClassTag + +/** + * Extends `RDD[(VertexId, VD)]` by ensuring that there is only one entry for each vertex and by + * pre-indexing the entries for fast, efficient joins. Two VertexRDDs with the same index can be + * joined efficiently. All operations except [[reindex]] preserve the index. To construct a + * `VertexRDD`, use the [[org.apache.spark.graphframes.graphx.VertexRDD$ VertexRDD object]]. + * + * Additionally, stores routing information to enable joining the vertex attributes with an + * [[org.apache.spark.graphframes.graphx.EdgeRDD]]. + * + * @example + * Construct a `VertexRDD` from a plain RDD: + * {{{ + * // Construct an initial vertex set + * val someData: RDD[(VertexId, SomeType)] = loadData(someFile) + * val vset = VertexRDD(someData) + * // If there were redundant values in someData we would use a reduceFunc + * val vset2 = VertexRDD(someData, reduceFunc) + * // Finally we can use the VertexRDD to index another dataset + * val otherData: RDD[(VertexId, OtherType)] = loadData(otherFile) + * val vset3 = vset2.innerJoin(otherData) { (vid, a, b) => b } + * // Now we can construct very fast joins between the two sets + * val vset4: VertexRDD[(SomeType, OtherType)] = vset.leftJoin(vset3) + * }}} + * @tparam VD + * the vertex attribute associated with each vertex in the set. + */ +abstract class VertexRDD[VD](sc: SparkContext, deps: Seq[Dependency[_]]) + extends RDD[(VertexId, VD)](sc, deps) { + + implicit protected def vdTag: ClassTag[VD] + + private[graphx] def partitionsRDD: RDD[ShippableVertexPartition[VD]] + + override protected def getPartitions: Array[Partition] = partitionsRDD.partitions + + /** + * Provides the `RDD[(VertexId, VD)]` equivalent output. + */ + override def compute(part: Partition, context: TaskContext): Iterator[(VertexId, VD)] = { + firstParent[ShippableVertexPartition[VD]].iterator(part, context).next().iterator + } + + /** + * Construct a new VertexRDD that is indexed by only the visible vertices. The resulting + * VertexRDD will be based on a different index and can no longer be quickly joined with this + * RDD. + */ + def reindex(): VertexRDD[VD] + + /** + * Applies a function to each `VertexPartition` of this RDD and returns a new VertexRDD. + */ + private[graphx] def mapVertexPartitions[VD2: ClassTag]( + f: ShippableVertexPartition[VD] => ShippableVertexPartition[VD2]): VertexRDD[VD2] + + /** + * Restricts the vertex set to the set of vertices satisfying the given predicate. This + * operation preserves the index for efficient joins with the original RDD, and it sets bits in + * the bitmask rather than allocating new memory. + * + * It is declared and defined here to allow refining the return type from `RDD[(VertexId, VD)]` + * to `VertexRDD[VD]`. + * + * @param pred + * the user defined predicate, which takes a tuple to conform to the `RDD[(VertexId, VD)]` + * interface + */ + override def filter(pred: Tuple2[VertexId, VD] => Boolean): VertexRDD[VD] = + this.mapVertexPartitions(_.filter(Function.untupled(pred))) + + /** + * Maps each vertex attribute, preserving the index. + * + * @tparam VD2 + * the type returned by the map function + * + * @param f + * the function applied to each value in the RDD + * @return + * a new VertexRDD with values obtained by applying `f` to each of the entries in the original + * VertexRDD + */ + def mapValues[VD2: ClassTag](f: VD => VD2): VertexRDD[VD2] + + /** + * Maps each vertex attribute, additionally supplying the vertex ID. + * + * @tparam VD2 + * the type returned by the map function + * + * @param f + * the function applied to each ID-value pair in the RDD + * @return + * a new VertexRDD with values obtained by applying `f` to each of the entries in the original + * VertexRDD. The resulting VertexRDD retains the same index. + */ + def mapValues[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexRDD[VD2] + + /** + * For each VertexId present in both `this` and `other`, minus will act as a set difference + * operation returning only those unique VertexId's present in `this`. + * + * @param other + * an RDD to run the set operation against + */ + def minus(other: RDD[(VertexId, VD)]): VertexRDD[VD] + + /** + * For each VertexId present in both `this` and `other`, minus will act as a set difference + * operation returning only those unique VertexId's present in `this`. + * + * @param other + * a VertexRDD to run the set operation against + */ + def minus(other: VertexRDD[VD]): VertexRDD[VD] + + /** + * For each vertex present in both `this` and `other`, `diff` returns only those vertices with + * differing values; for values that are different, keeps the values from `other`. This is only + * guaranteed to work if the VertexRDDs share a common ancestor. + * + * @param other + * the other RDD[(VertexId, VD)] with which to diff against. + */ + def diff(other: RDD[(VertexId, VD)]): VertexRDD[VD] + + /** + * For each vertex present in both `this` and `other`, `diff` returns only those vertices with + * differing values; for values that are different, keeps the values from `other`. This is only + * guaranteed to work if the VertexRDDs share a common ancestor. + * + * @param other + * the other VertexRDD with which to diff against. + */ + def diff(other: VertexRDD[VD]): VertexRDD[VD] + + /** + * Left joins this RDD with another VertexRDD with the same index. This function will fail if + * both VertexRDDs do not share the same index. The resulting vertex set contains an entry for + * each vertex in `this`. If `other` is missing any vertex in this VertexRDD, `f` is passed + * `None`. + * + * @tparam VD2 + * the attribute type of the other VertexRDD + * @tparam VD3 + * the attribute type of the resulting VertexRDD + * + * @param other + * the other VertexRDD with which to join. + * @param f + * the function mapping a vertex id and its attributes in this and the other vertex set to a + * new vertex attribute. + * @return + * a VertexRDD containing the results of `f` + */ + def leftZipJoin[VD2: ClassTag, VD3: ClassTag](other: VertexRDD[VD2])( + f: (VertexId, VD, Option[VD2]) => VD3): VertexRDD[VD3] + + /** + * Left joins this VertexRDD with an RDD containing vertex attribute pairs. If the other RDD is + * backed by a VertexRDD with the same index then the efficient [[leftZipJoin]] implementation + * is used. The resulting VertexRDD contains an entry for each vertex in `this`. If `other` is + * missing any vertex in this VertexRDD, `f` is passed `None`. If there are duplicates, the + * vertex is picked arbitrarily. + * + * @tparam VD2 + * the attribute type of the other VertexRDD + * @tparam VD3 + * the attribute type of the resulting VertexRDD + * + * @param other + * the other VertexRDD with which to join + * @param f + * the function mapping a vertex id and its attributes in this and the other vertex set to a + * new vertex attribute. + * @return + * a VertexRDD containing all the vertices in this VertexRDD with the attributes emitted by + * `f`. + */ + def leftJoin[VD2: ClassTag, VD3: ClassTag](other: RDD[(VertexId, VD2)])( + f: (VertexId, VD, Option[VD2]) => VD3): VertexRDD[VD3] + + /** + * Efficiently inner joins this VertexRDD with another VertexRDD sharing the same index. See + * [[innerJoin]] for the behavior of the join. + */ + def innerZipJoin[U: ClassTag, VD2: ClassTag](other: VertexRDD[U])( + f: (VertexId, VD, U) => VD2): VertexRDD[VD2] + + /** + * Inner joins this VertexRDD with an RDD containing vertex attribute pairs. If the other RDD is + * backed by a VertexRDD with the same index then the efficient [[innerZipJoin]] implementation + * is used. + * + * @param other + * an RDD containing vertices to join. If there are multiple entries for the same vertex, one + * is picked arbitrarily. Use [[aggregateUsingIndex]] to merge multiple entries. + * @param f + * the join function applied to corresponding values of `this` and `other` + * @return + * a VertexRDD co-indexed with `this`, containing only vertices that appear in both `this` and + * `other`, with values supplied by `f` + */ + def innerJoin[U: ClassTag, VD2: ClassTag](other: RDD[(VertexId, U)])( + f: (VertexId, VD, U) => VD2): VertexRDD[VD2] + + /** + * Aggregates vertices in `messages` that have the same ids using `reduceFunc`, returning a + * VertexRDD co-indexed with `this`. + * + * @param messages + * an RDD containing messages to aggregate, where each message is a pair of its target vertex + * ID and the message data + * @param reduceFunc + * the associative aggregation function for merging messages to the same vertex + * @return + * a VertexRDD co-indexed with `this`, containing only vertices that received messages. For + * those vertices, their values are the result of applying `reduceFunc` to all received + * messages. + */ + def aggregateUsingIndex[VD2: ClassTag]( + messages: RDD[(VertexId, VD2)], + reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] + + /** + * Returns a new `VertexRDD` reflecting a reversal of all edge directions in the corresponding + * [[EdgeRDD]]. + */ + def reverseRoutingTables(): VertexRDD[VD] + + /** Prepares this VertexRDD for efficient joins with the given EdgeRDD. */ + def withEdges(edges: EdgeRDD[_]): VertexRDD[VD] + + /** Replaces the vertex partitions while preserving all other properties of the VertexRDD. */ + private[graphx] def withPartitionsRDD[VD2: ClassTag]( + partitionsRDD: RDD[ShippableVertexPartition[VD2]]): VertexRDD[VD2] + + /** + * Changes the target storage level while preserving all other properties of the VertexRDD. + * Operations on the returned VertexRDD will preserve this storage level. + * + * This does not actually trigger a cache; to do this, call + * [[org.apache.spark.graphframes.graphx.VertexRDD#cache]] on the returned VertexRDD. + */ + private[graphx] def withTargetStorageLevel(targetStorageLevel: StorageLevel): VertexRDD[VD] + + /** Generates an RDD of vertex attributes suitable for shipping to the edge partitions. */ + private[graphx] def shipVertexAttributes( + shipSrc: Boolean, + shipDst: Boolean): RDD[(PartitionID, VertexAttributeBlock[VD])] + + /** Generates an RDD of vertex IDs suitable for shipping to the edge partitions. */ + private[graphx] def shipVertexIds(): RDD[(PartitionID, Array[VertexId])] + +} // end of VertexRDD + +/** + * The VertexRDD singleton is used to construct VertexRDDs. + */ +object VertexRDD { + + /** + * Constructs a standalone `VertexRDD` (one that is not set up for efficient joins with an + * [[EdgeRDD]]) from an RDD of vertex-attribute pairs. Duplicate entries are removed + * arbitrarily. + * + * @tparam VD + * the vertex attribute type + * + * @param vertices + * the collection of vertex-attribute pairs + */ + def apply[VD: ClassTag](vertices: RDD[(VertexId, VD)]): VertexRDD[VD] = { + val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match { + case Some(_) => vertices + case None => vertices.partitionBy(new HashPartitioner(vertices.partitions.length)) + } + val vertexPartitions = vPartitioned.mapPartitions( + iter => Iterator(ShippableVertexPartition(iter)), + preservesPartitioning = true) + new VertexRDDImpl(vertexPartitions) + } + + /** + * Constructs a `VertexRDD` from an RDD of vertex-attribute pairs. Duplicate vertex entries are + * removed arbitrarily. The resulting `VertexRDD` will be joinable with `edges`, and any missing + * vertices referred to by `edges` will be created with the attribute `defaultVal`. + * + * @tparam VD + * the vertex attribute type + * + * @param vertices + * the collection of vertex-attribute pairs + * @param edges + * the [[EdgeRDD]] that these vertices may be joined with + * @param defaultVal + * the vertex attribute to use when creating missing vertices + */ + def apply[VD: ClassTag]( + vertices: RDD[(VertexId, VD)], + edges: EdgeRDD[_], + defaultVal: VD): VertexRDD[VD] = { + VertexRDD(vertices, edges, defaultVal, (a, _) => a) + } + + /** + * Constructs a `VertexRDD` from an RDD of vertex-attribute pairs. Duplicate vertex entries are + * merged using `mergeFunc`. The resulting `VertexRDD` will be joinable with `edges`, and any + * missing vertices referred to by `edges` will be created with the attribute `defaultVal`. + * + * @tparam VD + * the vertex attribute type + * + * @param vertices + * the collection of vertex-attribute pairs + * @param edges + * the [[EdgeRDD]] that these vertices may be joined with + * @param defaultVal + * the vertex attribute to use when creating missing vertices + * @param mergeFunc + * the commutative, associative duplicate vertex attribute merge function + */ + def apply[VD: ClassTag]( + vertices: RDD[(VertexId, VD)], + edges: EdgeRDD[_], + defaultVal: VD, + mergeFunc: (VD, VD) => VD): VertexRDD[VD] = { + val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match { + case Some(_) => vertices + case None => vertices.partitionBy(new HashPartitioner(vertices.partitions.length)) + } + val routingTables = createRoutingTables(edges, vPartitioned.partitioner.get) + val vertexPartitions = + vPartitioned.zipPartitions(routingTables, preservesPartitioning = true) { + (vertexIter, routingTableIter) => + val routingTable = + if (routingTableIter.hasNext) routingTableIter.next() else RoutingTablePartition.empty + Iterator(ShippableVertexPartition(vertexIter, routingTable, defaultVal, mergeFunc)) + } + new VertexRDDImpl(vertexPartitions) + } + + /** + * Constructs a `VertexRDD` containing all vertices referred to in `edges`. The vertices will be + * created with the attribute `defaultVal`. The resulting `VertexRDD` will be joinable with + * `edges`. + * + * @tparam VD + * the vertex attribute type + * + * @param edges + * the [[EdgeRDD]] referring to the vertices to create + * @param numPartitions + * the desired number of partitions for the resulting `VertexRDD` + * @param defaultVal + * the vertex attribute to use when creating missing vertices + */ + def fromEdges[VD: ClassTag]( + edges: EdgeRDD[_], + numPartitions: Int, + defaultVal: VD): VertexRDD[VD] = { + val routingTables = createRoutingTables(edges, new HashPartitioner(numPartitions)) + val vertexPartitions = routingTables.mapPartitions( + { routingTableIter => + val routingTable = + if (routingTableIter.hasNext) routingTableIter.next() else RoutingTablePartition.empty + Iterator(ShippableVertexPartition(Iterator.empty, routingTable, defaultVal)) + }, + preservesPartitioning = true) + new VertexRDDImpl(vertexPartitions) + } + + private[graphx] def createRoutingTables( + edges: EdgeRDD[_], + vertexPartitioner: Partitioner): RDD[RoutingTablePartition] = { + // Determine which vertices each edge partition needs by creating a mapping from vid to pid. + val vid2pid = edges.partitionsRDD + .mapPartitions(_.flatMap(Function.tupled(RoutingTablePartition.edgePartitionToMsgs))) + .setName("VertexRDD.createRoutingTables - vid2pid (aggregation)") + + val numEdgePartitions = edges.partitions.length + vid2pid + .partitionBy(vertexPartitioner) + .mapPartitions( + iter => Iterator(RoutingTablePartition.fromMsgs(numEdgePartitions, iter)), + preservesPartitioning = true) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/EdgePartition.scala new file mode 100644 index 000000000..34ede5f35 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/EdgePartition.scala @@ -0,0 +1,613 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.impl + +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.graphframes.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.BitSet + +import scala.annotation.nowarn +import scala.reflect.ClassTag + +/** + * A collection of edges, along with referenced vertex attributes and an optional active vertex + * set for filtering computation on the edges. + * + * The edges are stored in columnar format in `localSrcIds`, `localDstIds`, and `data`. All + * referenced global vertex ids are mapped to a compact set of local vertex ids according to the + * `global2local` map. Each local vertex id is a valid index into `vertexAttrs`, which stores the + * corresponding vertex attribute, and `local2global`, which stores the reverse mapping to global + * vertex id. The global vertex ids that are active are optionally stored in `activeSet`. + * + * The edges are clustered by source vertex id, and the mapping from global vertex id to the index + * of the corresponding edge cluster is stored in `index`. + * + * @tparam ED + * the edge attribute type + * @tparam VD + * the vertex attribute type + * + * @param localSrcIds + * the local source vertex id of each edge as an index into `local2global` and `vertexAttrs` + * @param localDstIds + * the local destination vertex id of each edge as an index into `local2global` and + * `vertexAttrs` + * @param data + * the attribute associated with each edge + * @param index + * a clustered index on source vertex id as a map from each global source vertex id to the + * offset in the edge arrays where the cluster for that vertex id begins + * @param global2local + * a map from referenced vertex ids to local ids which index into vertexAttrs + * @param local2global + * an array of global vertex ids where the offsets are local vertex ids + * @param vertexAttrs + * an array of vertex attributes where the offsets are local vertex ids + * @param activeSet + * an optional active vertex set for filtering computation on the edges + */ +private[graphx] class EdgePartition[ + @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag, + VD: ClassTag]( + localSrcIds: Array[Int], + localDstIds: Array[Int], + data: Array[ED], + index: GraphXPrimitiveKeyOpenHashMap[VertexId, Int], + global2local: GraphXPrimitiveKeyOpenHashMap[VertexId, Int], + local2global: Array[VertexId], + vertexAttrs: Array[VD], + activeSet: Option[VertexSet]) + extends Serializable { + + /** No-arg constructor for serialization. */ + @nowarn + private def this() = this(null, null, null, null, null, null, null, null) + + /** Return a new `EdgePartition` with the specified edge data. */ + def withData[ED2: ClassTag](data: Array[ED2]): EdgePartition[ED2, VD] = { + new EdgePartition( + localSrcIds, + localDstIds, + data, + index, + global2local, + local2global, + vertexAttrs, + activeSet) + } + + /** Return a new `EdgePartition` with the specified active set, provided as an iterator. */ + def withActiveSet(iter: Iterator[VertexId]): EdgePartition[ED, VD] = { + val activeSet = new VertexSet + while (iter.hasNext) { activeSet.add(iter.next()) } + new EdgePartition( + localSrcIds, + localDstIds, + data, + index, + global2local, + local2global, + vertexAttrs, + Some(activeSet)) + } + + /** Return a new `EdgePartition` with updates to vertex attributes specified in `iter`. */ + def updateVertices(iter: Iterator[(VertexId, VD)]): EdgePartition[ED, VD] = { + val newVertexAttrs = new Array[VD](vertexAttrs.length) + System.arraycopy(vertexAttrs, 0, newVertexAttrs, 0, vertexAttrs.length) + while (iter.hasNext) { + val kv = iter.next() + newVertexAttrs(global2local(kv._1)) = kv._2 + } + new EdgePartition( + localSrcIds, + localDstIds, + data, + index, + global2local, + local2global, + newVertexAttrs, + activeSet) + } + + /** Return a new `EdgePartition` without any locally cached vertex attributes. */ + def withoutVertexAttributes[VD2: ClassTag](): EdgePartition[ED, VD2] = { + val newVertexAttrs = new Array[VD2](vertexAttrs.length) + new EdgePartition( + localSrcIds, + localDstIds, + data, + index, + global2local, + local2global, + newVertexAttrs, + activeSet) + } + + @inline private def srcIds(pos: Int): VertexId = local2global(localSrcIds(pos)) + + @inline private def dstIds(pos: Int): VertexId = local2global(localDstIds(pos)) + + @inline private def attrs(pos: Int): ED = data(pos) + + /** Look up vid in activeSet, throwing an exception if it is None. */ + def isActive(vid: VertexId): Boolean = { + activeSet.get.contains(vid) + } + + /** The number of active vertices, if any exist. */ + def numActives: Option[Int] = activeSet.map(_.size) + + /** + * Reverse all the edges in this partition. + * + * @return + * a new edge partition with all edges reversed. + */ + def reverse: EdgePartition[ED, VD] = { + val builder = new ExistingEdgePartitionBuilder[ED, VD]( + global2local, + local2global, + vertexAttrs, + activeSet, + size) + var i = 0 + while (i < size) { + val localSrcId = localSrcIds(i) + val localDstId = localDstIds(i) + val srcId = local2global(localSrcId) + val dstId = local2global(localDstId) + val attr = data(i) + builder.add(dstId, srcId, localDstId, localSrcId, attr) + i += 1 + } + builder.toEdgePartition + } + + /** + * Construct a new edge partition by applying the function f to all edges in this partition. + * + * Be careful not to keep references to the objects passed to `f`. To improve GC performance the + * same object is re-used for each call. + * + * @param f + * a function from an edge to a new attribute + * @tparam ED2 + * the type of the new attribute + * @return + * a new edge partition with the result of the function `f` applied to each edge + */ + def map[ED2: ClassTag](f: Edge[ED] => ED2): EdgePartition[ED2, VD] = { + val newData = new Array[ED2](data.length) + val edge = new Edge[ED]() + val size = data.length + var i = 0 + while (i < size) { + edge.srcId = srcIds(i) + edge.dstId = dstIds(i) + edge.attr = data(i) + newData(i) = f(edge) + i += 1 + } + this.withData(newData) + } + + /** + * Construct a new edge partition by using the edge attributes contained in the iterator. + * + * @note + * The input iterator should return edge attributes in the order of the edges returned by + * `EdgePartition.iterator` and should return attributes equal to the number of edges. + * + * @param iter + * an iterator for the new attribute values + * @tparam ED2 + * the type of the new attribute + * @return + * a new edge partition with the attribute values replaced + */ + def map[ED2: ClassTag](iter: Iterator[ED2]): EdgePartition[ED2, VD] = { + // Faster than iter.toArray, because the expected size is known. + val newData = new Array[ED2](data.length) + var i = 0 + while (iter.hasNext) { + newData(i) = iter.next() + i += 1 + } + assert(newData.length == i) + this.withData(newData) + } + + /** + * Construct a new edge partition containing only the edges matching `epred` and where both + * vertices match `vpred`. + */ + def filter( + epred: EdgeTriplet[VD, ED] => Boolean, + vpred: (VertexId, VD) => Boolean): EdgePartition[ED, VD] = { + val builder = + new ExistingEdgePartitionBuilder[ED, VD](global2local, local2global, vertexAttrs, activeSet) + var i = 0 + while (i < size) { + // The user sees the EdgeTriplet, so we can't reuse it and must create one per edge. + val localSrcId = localSrcIds(i) + val localDstId = localDstIds(i) + val et = new EdgeTriplet[VD, ED] + et.srcId = local2global(localSrcId) + et.dstId = local2global(localDstId) + et.srcAttr = vertexAttrs(localSrcId) + et.dstAttr = vertexAttrs(localDstId) + et.attr = data(i) + if (vpred(et.srcId, et.srcAttr) && vpred(et.dstId, et.dstAttr) && epred(et)) { + builder.add(et.srcId, et.dstId, localSrcId, localDstId, et.attr) + } + i += 1 + } + builder.toEdgePartition + } + + /** + * Apply the function f to all edges in this partition. + * + * @param f + * an external state mutating user defined function. + */ + def foreach(f: Edge[ED] => Unit): Unit = { + iterator.foreach(f) + } + + /** + * Merge all the edges with the same src and dest id into a single edge using the `merge` + * function + * + * @param merge + * a commutative associative merge operation + * @return + * a new edge partition without duplicate edges + */ + def groupEdges(merge: (ED, ED) => ED): EdgePartition[ED, VD] = { + val builder = + new ExistingEdgePartitionBuilder[ED, VD](global2local, local2global, vertexAttrs, activeSet) + var currSrcId: VertexId = null.asInstanceOf[VertexId] + var currDstId: VertexId = null.asInstanceOf[VertexId] + var currLocalSrcId = -1 + var currLocalDstId = -1 + var currAttr: ED = null.asInstanceOf[ED] + // Iterate through the edges, accumulating runs of identical edges using the curr* variables and + // releasing them to the builder when we see the beginning of the next run + var i = 0 + while (i < size) { + if (i > 0 && currSrcId == srcIds(i) && currDstId == dstIds(i)) { + // This edge should be accumulated into the existing run + currAttr = merge(currAttr, data(i)) + } else { + // This edge starts a new run of edges + if (i > 0) { + // First release the existing run to the builder + builder.add(currSrcId, currDstId, currLocalSrcId, currLocalDstId, currAttr) + } + // Then start accumulating for a new run + currSrcId = srcIds(i) + currDstId = dstIds(i) + currLocalSrcId = localSrcIds(i) + currLocalDstId = localDstIds(i) + currAttr = data(i) + } + i += 1 + } + // Finally, release the last accumulated run + if (size > 0) { + builder.add(currSrcId, currDstId, currLocalSrcId, currLocalDstId, currAttr) + } + builder.toEdgePartition + } + + /** + * Apply `f` to all edges present in both `this` and `other` and return a new `EdgePartition` + * containing the resulting edges. + * + * If there are multiple edges with the same src and dst in `this`, `f` will be invoked once for + * each edge, but each time it may be invoked on any corresponding edge in `other`. + * + * If there are multiple edges with the same src and dst in `other`, `f` will only be invoked + * once. + */ + @nowarn + def innerJoin[ED2: ClassTag, ED3: ClassTag](other: EdgePartition[ED2, _])( + f: (VertexId, VertexId, ED, ED2) => ED3): EdgePartition[ED3, VD] = { + val builder = new ExistingEdgePartitionBuilder[ED3, VD]( + global2local, + local2global, + vertexAttrs, + activeSet) + var i = 0 + var j = 0 + // For i = index of each edge in `this`... + while (i < size && j < other.size) { + val srcId = this.srcIds(i) + val dstId = this.dstIds(i) + // ... forward j to the index of the corresponding edge in `other`, and... + while (j < other.size && other.srcIds(j) < srcId) { j += 1 } + if (j < other.size && other.srcIds(j) == srcId) { + while (j < other.size && other.srcIds(j) == srcId && other.dstIds(j) < dstId) { j += 1 } + if (j < other.size && other.srcIds(j) == srcId && other.dstIds(j) == dstId) { + // ... run `f` on the matching edge + builder.add( + srcId, + dstId, + localSrcIds(i), + localDstIds(i), + f(srcId, dstId, this.data(i), other.attrs(j))) + } + } + i += 1 + } + builder.toEdgePartition + } + + /** + * The number of edges in this partition + * + * @return + * size of the partition + */ + val size: Int = localSrcIds.length + + /** The number of unique source vertices in the partition. */ + def indexSize: Int = index.size + + /** + * Get an iterator over the edges in this partition. + * + * Be careful not to keep references to the objects from this iterator. To improve GC + * performance the same object is re-used in `next()`. + * + * @return + * an iterator over edges in the partition + */ + def iterator: Iterator[Edge[ED]] = new Iterator[Edge[ED]] { + private[this] val edge = new Edge[ED] + private[this] var pos = 0 + + override def hasNext: Boolean = pos < EdgePartition.this.size + + override def next(): Edge[ED] = { + edge.srcId = srcIds(pos) + edge.dstId = dstIds(pos) + edge.attr = data(pos) + pos += 1 + edge + } + } + + /** + * Get an iterator over the edge triplets in this partition. + * + * It is safe to keep references to the objects from this iterator. + */ + def tripletIterator( + includeSrc: Boolean = true, + includeDst: Boolean = true): Iterator[EdgeTriplet[VD, ED]] = + new Iterator[EdgeTriplet[VD, ED]] { + private[this] var pos = 0 + + override def hasNext: Boolean = pos < EdgePartition.this.size + + override def next(): EdgeTriplet[VD, ED] = { + val triplet = new EdgeTriplet[VD, ED] + val localSrcId = localSrcIds(pos) + val localDstId = localDstIds(pos) + triplet.srcId = local2global(localSrcId) + triplet.dstId = local2global(localDstId) + if (includeSrc) { + triplet.srcAttr = vertexAttrs(localSrcId) + } + if (includeDst) { + triplet.dstAttr = vertexAttrs(localDstId) + } + triplet.attr = data(pos) + pos += 1 + triplet + } + } + + /** + * Send messages along edges and aggregate them at the receiving vertices. Implemented by + * scanning all edges sequentially. + * + * @param sendMsg + * generates messages to neighboring vertices of an edge + * @param mergeMsg + * the combiner applied to messages destined to the same vertex + * @param tripletFields + * which triplet fields `sendMsg` uses + * @param activeness + * criteria for filtering edges based on activeness + * + * @return + * iterator aggregated messages keyed by the receiving vertex id + */ + def aggregateMessagesEdgeScan[A: ClassTag]( + sendMsg: EdgeContext[VD, ED, A] => Unit, + mergeMsg: (A, A) => A, + tripletFields: TripletFields, + activeness: EdgeActiveness): Iterator[(VertexId, A)] = { + val aggregates = new Array[A](vertexAttrs.length) + val bitset = new BitSet(vertexAttrs.length) + + val ctx = new AggregatingEdgeContext[VD, ED, A](mergeMsg, aggregates, bitset) + var i = 0 + while (i < size) { + val localSrcId = localSrcIds(i) + val srcId = local2global(localSrcId) + val localDstId = localDstIds(i) + val dstId = local2global(localDstId) + val edgeIsActive = + if (activeness == EdgeActiveness.Neither) true + else if (activeness == EdgeActiveness.SrcOnly) isActive(srcId) + else if (activeness == EdgeActiveness.DstOnly) isActive(dstId) + else if (activeness == EdgeActiveness.Both) isActive(srcId) && isActive(dstId) + else if (activeness == EdgeActiveness.Either) isActive(srcId) || isActive(dstId) + else throw new Exception("unreachable") + if (edgeIsActive) { + val srcAttr = if (tripletFields.useSrc) vertexAttrs(localSrcId) else null.asInstanceOf[VD] + val dstAttr = if (tripletFields.useDst) vertexAttrs(localDstId) else null.asInstanceOf[VD] + ctx.set(srcId, dstId, localSrcId, localDstId, srcAttr, dstAttr, data(i)) + sendMsg(ctx) + } + i += 1 + } + + bitset.iterator.map { localId => (local2global(localId), aggregates(localId)) } + } + + /** + * Send messages along edges and aggregate them at the receiving vertices. Implemented by + * filtering the source vertex index, then scanning each edge cluster. + * + * @param sendMsg + * generates messages to neighboring vertices of an edge + * @param mergeMsg + * the combiner applied to messages destined to the same vertex + * @param tripletFields + * which triplet fields `sendMsg` uses + * @param activeness + * criteria for filtering edges based on activeness + * + * @return + * iterator aggregated messages keyed by the receiving vertex id + */ + def aggregateMessagesIndexScan[A: ClassTag]( + sendMsg: EdgeContext[VD, ED, A] => Unit, + mergeMsg: (A, A) => A, + tripletFields: TripletFields, + activeness: EdgeActiveness): Iterator[(VertexId, A)] = { + val aggregates = new Array[A](vertexAttrs.length) + val bitset = new BitSet(vertexAttrs.length) + + val ctx = new AggregatingEdgeContext[VD, ED, A](mergeMsg, aggregates, bitset) + index.iterator.foreach { cluster => + val clusterSrcId = cluster._1 + val clusterPos = cluster._2 + val clusterLocalSrcId = localSrcIds(clusterPos) + + val scanCluster = + if (activeness == EdgeActiveness.Neither) true + else if (activeness == EdgeActiveness.SrcOnly) isActive(clusterSrcId) + else if (activeness == EdgeActiveness.DstOnly) true + else if (activeness == EdgeActiveness.Both) isActive(clusterSrcId) + else if (activeness == EdgeActiveness.Either) true + else throw new Exception("unreachable") + + if (scanCluster) { + var pos = clusterPos + val srcAttr = + if (tripletFields.useSrc) vertexAttrs(clusterLocalSrcId) else null.asInstanceOf[VD] + ctx.setSrcOnly(clusterSrcId, clusterLocalSrcId, srcAttr) + while (pos < size && localSrcIds(pos) == clusterLocalSrcId) { + val localDstId = localDstIds(pos) + val dstId = local2global(localDstId) + val edgeIsActive = + if (activeness == EdgeActiveness.Neither) true + else if (activeness == EdgeActiveness.SrcOnly) true + else if (activeness == EdgeActiveness.DstOnly) isActive(dstId) + else if (activeness == EdgeActiveness.Both) isActive(dstId) + else if (activeness == EdgeActiveness.Either) + isActive(clusterSrcId) || isActive(dstId) + else throw new Exception("unreachable") + if (edgeIsActive) { + val dstAttr = + if (tripletFields.useDst) vertexAttrs(localDstId) else null.asInstanceOf[VD] + ctx.setDest(dstId, localDstId, dstAttr, data(pos)) + sendMsg(ctx) + } + pos += 1 + } + } + } + + bitset.iterator.map { localId => (local2global(localId), aggregates(localId)) } + } +} + +private class AggregatingEdgeContext[VD, ED, A]( + mergeMsg: (A, A) => A, + aggregates: Array[A], + bitset: BitSet) + extends EdgeContext[VD, ED, A] { + + private[this] var _srcId: VertexId = _ + private[this] var _dstId: VertexId = _ + private[this] var _localSrcId: Int = _ + private[this] var _localDstId: Int = _ + private[this] var _srcAttr: VD = _ + private[this] var _dstAttr: VD = _ + private[this] var _attr: ED = _ + + def set( + srcId: VertexId, + dstId: VertexId, + localSrcId: Int, + localDstId: Int, + srcAttr: VD, + dstAttr: VD, + attr: ED): Unit = { + _srcId = srcId + _dstId = dstId + _localSrcId = localSrcId + _localDstId = localDstId + _srcAttr = srcAttr + _dstAttr = dstAttr + _attr = attr + } + + def setSrcOnly(srcId: VertexId, localSrcId: Int, srcAttr: VD): Unit = { + _srcId = srcId + _localSrcId = localSrcId + _srcAttr = srcAttr + } + + def setDest(dstId: VertexId, localDstId: Int, dstAttr: VD, attr: ED): Unit = { + _dstId = dstId + _localDstId = localDstId + _dstAttr = dstAttr + _attr = attr + } + + override def srcId: VertexId = _srcId + override def dstId: VertexId = _dstId + override def srcAttr: VD = _srcAttr + override def dstAttr: VD = _dstAttr + override def attr: ED = _attr + + override def sendToSrc(msg: A): Unit = { + send(_localSrcId, msg) + } + override def sendToDst(msg: A): Unit = { + send(_localDstId, msg) + } + + @inline private def send(localId: Int, msg: A): Unit = { + if (bitset.get(localId)) { + aggregates(localId) = mergeMsg(aggregates(localId), msg) + } else { + aggregates(localId) = msg + bitset.set(localId) + } + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/EdgePartitionBuilder.scala new file mode 100644 index 000000000..d7f1d1cef --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/EdgePartitionBuilder.scala @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.impl + +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.graphframes.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.PrimitiveVector +import org.apache.spark.util.collection.SortDataFormat +import org.apache.spark.util.collection.Sorter + +import scala.reflect.ClassTag + +/** Constructs an EdgePartition from scratch. */ +private[graphx] class EdgePartitionBuilder[ + @specialized(Long, Int, Double) ED: ClassTag, + VD: ClassTag](size: Int = 64) { + private[this] val edges = new PrimitiveVector[Edge[ED]](size) + + /** Add a new edge to the partition. */ + def add(src: VertexId, dst: VertexId, d: ED): Unit = { + edges += Edge(src, dst, d) + } + + def toEdgePartition: EdgePartition[ED, VD] = { + val edgeArray = edges.trim().array + new Sorter(Edge.edgeArraySortDataFormat[ED]) + .sort(edgeArray, 0, edgeArray.length, Edge.lexicographicOrdering) + val localSrcIds = new Array[Int](edgeArray.length) + val localDstIds = new Array[Int](edgeArray.length) + val data = new Array[ED](edgeArray.length) + val index = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int] + val global2local = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int] + val local2global = new PrimitiveVector[VertexId] + var vertexAttrs = Array.empty[VD] + // Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and + // adding them to the index. Also populate a map from vertex id to a sequential local offset. + if (edgeArray.length > 0) { + index.update(edgeArray(0).srcId, 0) + var currSrcId: VertexId = edgeArray(0).srcId + var currLocalId = -1 + var i = 0 + while (i < edgeArray.length) { + val srcId = edgeArray(i).srcId + val dstId = edgeArray(i).dstId + localSrcIds(i) = global2local.changeValue( + srcId, + { currLocalId += 1; local2global += srcId; currLocalId }, + identity) + localDstIds(i) = global2local.changeValue( + dstId, + { currLocalId += 1; local2global += dstId; currLocalId }, + identity) + data(i) = edgeArray(i).attr + if (srcId != currSrcId) { + currSrcId = srcId + index.update(currSrcId, i) + } + + i += 1 + } + vertexAttrs = new Array[VD](currLocalId + 1) + } + new EdgePartition( + localSrcIds, + localDstIds, + data, + index, + global2local, + local2global.trim().array, + vertexAttrs, + None) + } +} + +/** + * Constructs an EdgePartition from an existing EdgePartition with the same vertex set. This + * enables reuse of the local vertex ids. Intended for internal use in EdgePartition only. + */ +private[impl] class ExistingEdgePartitionBuilder[ + @specialized(Long, Int, Double) ED: ClassTag, + VD: ClassTag]( + global2local: GraphXPrimitiveKeyOpenHashMap[VertexId, Int], + local2global: Array[VertexId], + vertexAttrs: Array[VD], + activeSet: Option[VertexSet], + size: Int = 64) { + private[this] val edges = new PrimitiveVector[EdgeWithLocalIds[ED]](size) + + /** Add a new edge to the partition. */ + def add(src: VertexId, dst: VertexId, localSrc: Int, localDst: Int, d: ED): Unit = { + edges += EdgeWithLocalIds(src, dst, localSrc, localDst, d) + } + + def toEdgePartition: EdgePartition[ED, VD] = { + val edgeArray = edges.trim().array + new Sorter(EdgeWithLocalIds.edgeArraySortDataFormat[ED]) + .sort(edgeArray, 0, edgeArray.length, EdgeWithLocalIds.lexicographicOrdering) + val localSrcIds = new Array[Int](edgeArray.length) + val localDstIds = new Array[Int](edgeArray.length) + val data = new Array[ED](edgeArray.length) + val index = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int] + // Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and + // adding them to the index + if (edgeArray.length > 0) { + index.update(edgeArray(0).srcId, 0) + var currSrcId: VertexId = edgeArray(0).srcId + var i = 0 + while (i < edgeArray.length) { + localSrcIds(i) = edgeArray(i).localSrcId + localDstIds(i) = edgeArray(i).localDstId + data(i) = edgeArray(i).attr + if (edgeArray(i).srcId != currSrcId) { + currSrcId = edgeArray(i).srcId + index.update(currSrcId, i) + } + i += 1 + } + } + + new EdgePartition( + localSrcIds, + localDstIds, + data, + index, + global2local, + local2global, + vertexAttrs, + activeSet) + } +} + +private[impl] case class EdgeWithLocalIds[@specialized ED]( + srcId: VertexId, + dstId: VertexId, + localSrcId: Int, + localDstId: Int, + attr: ED) + +private[impl] object EdgeWithLocalIds { + implicit def lexicographicOrdering[ED]: Ordering[EdgeWithLocalIds[ED]] = + (a: EdgeWithLocalIds[ED], b: EdgeWithLocalIds[ED]) => + if (a.srcId == b.srcId) { + if (a.dstId == b.dstId) 0 + else if (a.dstId < b.dstId) -1 + else 1 + } else if (a.srcId < b.srcId) -1 + else 1 + + private[graphx] def edgeArraySortDataFormat[ED] = { + new SortDataFormat[EdgeWithLocalIds[ED], Array[EdgeWithLocalIds[ED]]] { + override def getKey(data: Array[EdgeWithLocalIds[ED]], pos: Int): EdgeWithLocalIds[ED] = { + data(pos) + } + + override def swap(data: Array[EdgeWithLocalIds[ED]], pos0: Int, pos1: Int): Unit = { + val tmp = data(pos0) + data(pos0) = data(pos1) + data(pos1) = tmp + } + + override def copyElement( + src: Array[EdgeWithLocalIds[ED]], + srcPos: Int, + dst: Array[EdgeWithLocalIds[ED]], + dstPos: Int): Unit = { + dst(dstPos) = src(srcPos) + } + + override def copyRange( + src: Array[EdgeWithLocalIds[ED]], + srcPos: Int, + dst: Array[EdgeWithLocalIds[ED]], + dstPos: Int, + length: Int): Unit = { + System.arraycopy(src, srcPos, dst, dstPos, length) + } + + override def allocate(length: Int): Array[EdgeWithLocalIds[ED]] = { + new Array[EdgeWithLocalIds[ED]](length) + } + } + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/EdgeRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/EdgeRDDImpl.scala new file mode 100644 index 000000000..214d2645b --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/EdgeRDDImpl.scala @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.impl + +import org.apache.spark.HashPartitioner +import org.apache.spark.OneToOneDependency +import org.apache.spark.Partitioner +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + +import scala.annotation.nowarn +import scala.reflect.ClassTag +import scala.reflect.classTag + +class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( + @transient override val partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])], + val targetStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) + extends EdgeRDD[ED](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { + + override def setName(_name: String): this.type = { + if (partitionsRDD.name != null) { + partitionsRDD.setName(partitionsRDD.name + ", " + _name) + } else { + partitionsRDD.setName(_name) + } + this + } + setName("EdgeRDD") + + /** + * If `partitionsRDD` already has a partitioner, use it. Otherwise assume that the + * `PartitionID`s in `partitionsRDD` correspond to the actual partitions and create a new + * partitioner that allows co-partitioning with `partitionsRDD`. + */ + override val partitioner: Option[Partitioner] = + partitionsRDD.partitioner.orElse(Some(new HashPartitioner(partitions.length))) + + override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect() + + /** + * Persists the edge partitions at the specified storage level, ignoring any existing target + * storage level. + */ + override def persist(newLevel: StorageLevel): this.type = { + partitionsRDD.persist(newLevel) + this + } + + override def unpersist(blocking: Boolean = false): this.type = { + partitionsRDD.unpersist(blocking) + this + } + + /** + * Persists the edge partitions using `targetStorageLevel`, which defaults to MEMORY_ONLY. + */ + override def cache(): this.type = { + partitionsRDD.persist(targetStorageLevel) + this + } + + override def getStorageLevel: StorageLevel = partitionsRDD.getStorageLevel + + override def checkpoint(): Unit = { + partitionsRDD.checkpoint() + } + + override def isCheckpointed: Boolean = { + firstParent[(PartitionID, EdgePartition[ED, VD])].isCheckpointed + } + + override def getCheckpointFile: Option[String] = { + partitionsRDD.getCheckpointFile + } + + /** The number of edges in the RDD. */ + override def count(): Long = { + partitionsRDD.map(_._2.size.toLong).fold(0)(_ + _) + } + + override def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDDImpl[ED2, VD] = + mapEdgePartitions((_, part) => part.map(f)) + + override def reverse: EdgeRDDImpl[ED, VD] = mapEdgePartitions((_, part) => part.reverse) + + def filter( + epred: EdgeTriplet[VD, ED] => Boolean, + vpred: (VertexId, VD) => Boolean): EdgeRDDImpl[ED, VD] = { + mapEdgePartitions((_, part) => part.filter(epred, vpred)) + } + + override def innerJoin[ED2: ClassTag, ED3: ClassTag](other: EdgeRDD[ED2])( + f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDDImpl[ED3, VD] = { + val ed2Tag = classTag[ED2] + val ed3Tag = classTag[ED3] + this.withPartitionsRDD[ED3, VD](partitionsRDD.zipPartitions(other.partitionsRDD, true) { + (thisIter, otherIter) => + val (pid, thisEPart) = thisIter.next() + val (_, otherEPart) = otherIter.next() + Iterator(Tuple2(pid, thisEPart.innerJoin(otherEPart)(f)(ed2Tag, ed3Tag))) + }) + } + + @nowarn + def mapEdgePartitions[ED2: ClassTag, VD2: ClassTag]( + f: (PartitionID, EdgePartition[ED, VD]) => EdgePartition[ED2, VD2]) + : EdgeRDDImpl[ED2, VD2] = { + this.withPartitionsRDD[ED2, VD2]( + partitionsRDD.mapPartitions( + { iter => + if (iter.hasNext) { + val (pid, ep) = iter.next() + Iterator(Tuple2(pid, f(pid, ep))) + } else { + Iterator.empty + } + }, + preservesPartitioning = true)) + } + + private[graphx] def withPartitionsRDD[ED2: ClassTag, VD2: ClassTag]( + partitionsRDD: RDD[(PartitionID, EdgePartition[ED2, VD2])]): EdgeRDDImpl[ED2, VD2] = { + new EdgeRDDImpl(partitionsRDD, this.targetStorageLevel) + } + + override private[graphx] def withTargetStorageLevel( + targetStorageLevel: StorageLevel): EdgeRDDImpl[ED, VD] = { + new EdgeRDDImpl(this.partitionsRDD, targetStorageLevel) + } + +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/GraphImpl.scala new file mode 100644 index 000000000..7f8d79ca8 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/GraphImpl.scala @@ -0,0 +1,380 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.impl + +import org.apache.spark.HashPartitioner +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + +import scala.reflect.ClassTag +import scala.reflect.classTag + +/** + * An implementation of [[org.apache.spark.graphframes.graphx.Graph]] to support computation on + * graphs. + * + * Graphs are represented using two RDDs: `vertices`, which contains vertex attributes and the + * routing information for shipping vertex attributes to edge partitions, and + * `replicatedVertexView`, which contains edges and the vertex attributes mentioned by each edge. + */ +class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( + @transient val vertices: VertexRDD[VD], + @transient val replicatedVertexView: ReplicatedVertexView[VD, ED]) + extends Graph[VD, ED] + with Serializable { + + /** Default constructor is provided to support serialization */ + protected def this() = this(null, null) + + @transient override val edges: EdgeRDDImpl[ED, VD] = replicatedVertexView.edges + + /** Return an RDD that brings edges together with their source and destination vertices. */ + @transient override lazy val triplets: RDD[EdgeTriplet[VD, ED]] = { + replicatedVertexView.upgrade(vertices, true, true) + replicatedVertexView.edges.partitionsRDD.mapPartitions(_.flatMap { case (_, part) => + part.tripletIterator() + }) + } + + override def persist(newLevel: StorageLevel): Graph[VD, ED] = { + vertices.persist(newLevel) + replicatedVertexView.edges.persist(newLevel) + this + } + + override def cache(): Graph[VD, ED] = { + vertices.cache() + replicatedVertexView.edges.cache() + this + } + + override def checkpoint(): Unit = { + vertices.checkpoint() + replicatedVertexView.edges.checkpoint() + } + + override def isCheckpointed: Boolean = { + vertices.isCheckpointed && replicatedVertexView.edges.isCheckpointed + } + + override def getCheckpointFiles: Seq[String] = { + Seq(vertices.getCheckpointFile, replicatedVertexView.edges.getCheckpointFile).flatMap { + case Some(path) => Seq(path) + case None => Seq.empty + } + } + + override def unpersist(blocking: Boolean = false): Graph[VD, ED] = { + unpersistVertices(blocking) + replicatedVertexView.edges.unpersist(blocking) + this + } + + override def unpersistVertices(blocking: Boolean = false): Graph[VD, ED] = { + vertices.unpersist(blocking) + // TODO: unpersist the replicated vertices in `replicatedVertexView` but leave the edges alone + this + } + + override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { + partitionBy(partitionStrategy, edges.partitions.length) + } + + override def partitionBy( + partitionStrategy: PartitionStrategy, + numPartitions: Int): Graph[VD, ED] = { + val edTag = classTag[ED] + val vdTag = classTag[VD] + val newEdges = edges + .withPartitionsRDD( + edges + .map { e => + val part: PartitionID = + partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) + (part, (e.srcId, e.dstId, e.attr)) + } + .partitionBy(new HashPartitioner(numPartitions)) + .mapPartitionsWithIndex( + { (pid: Int, iter: Iterator[(PartitionID, (VertexId, VertexId, ED))]) => + val builder = new EdgePartitionBuilder[ED, VD]()(edTag, vdTag) + iter.foreach { message => + val data = message._2 + builder.add(data._1, data._2, data._3) + } + val edgePartition = builder.toEdgePartition + Iterator((pid, edgePartition)) + }, + preservesPartitioning = true)) + .cache() + GraphImpl.fromExistingRDDs(vertices.withEdges(newEdges), newEdges) + } + + override def reverse: Graph[VD, ED] = { + new GraphImpl(vertices.reverseRoutingTables(), replicatedVertexView.reverse()) + } + + override def mapVertices[VD2: ClassTag](f: (VertexId, VD) => VD2)(implicit + eq: VD =:= VD2 = null): Graph[VD2, ED] = { + // The implicit parameter eq will be populated by the compiler if VD and VD2 are equal, and left + // null if not + if (eq != null) { + vertices.cache() + // The map preserves type, so we can use incremental replication + val newVerts = vertices.mapVertexPartitions(_.map(f)).cache() + val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) + val newReplicatedVertexView = replicatedVertexView + .asInstanceOf[ReplicatedVertexView[VD2, ED]] + .updateVertices(changedVerts) + new GraphImpl(newVerts, newReplicatedVertexView) + } else { + // The map does not preserve type, so we must re-replicate all vertices + GraphImpl(vertices.mapVertexPartitions(_.map(f)), replicatedVertexView.edges) + } + } + + override def mapEdges[ED2: ClassTag]( + f: (PartitionID, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] = { + val newEdges = replicatedVertexView.edges + .mapEdgePartitions((pid, part) => part.map(f(pid, part.iterator))) + new GraphImpl(vertices, replicatedVertexView.withEdges(newEdges)) + } + + override def mapTriplets[ED2: ClassTag]( + f: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2], + tripletFields: TripletFields): Graph[VD, ED2] = { + vertices.cache() + replicatedVertexView.upgrade(vertices, tripletFields.useSrc, tripletFields.useDst) + val newEdges = replicatedVertexView.edges.mapEdgePartitions { (pid, part) => + part.map(f(pid, part.tripletIterator(tripletFields.useSrc, tripletFields.useDst))) + } + new GraphImpl(vertices, replicatedVertexView.withEdges(newEdges)) + } + + override def subgraph( + epred: EdgeTriplet[VD, ED] => Boolean = _ => true, + vpred: (VertexId, VD) => Boolean = (_, _) => true): Graph[VD, ED] = { + vertices.cache() + // Filter the vertices, reusing the partitioner and the index from this graph + val newVerts = vertices.mapVertexPartitions(_.filter(vpred)) + // Filter the triplets. We must always upgrade the triplet view fully because vpred always runs + // on both src and dst vertices + replicatedVertexView.upgrade(vertices, true, true) + val newEdges = replicatedVertexView.edges.filter(epred, vpred) + new GraphImpl(newVerts, replicatedVertexView.withEdges(newEdges)) + } + + override def mask[VD2: ClassTag, ED2: ClassTag](other: Graph[VD2, ED2]): Graph[VD, ED] = { + val newVerts = vertices.innerJoin(other.vertices) { (_, v, _) => v } + val newEdges = replicatedVertexView.edges.innerJoin(other.edges) { (_, _, v, _) => v } + new GraphImpl(newVerts, replicatedVertexView.withEdges(newEdges)) + } + + override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { + val newEdges = + replicatedVertexView.edges.mapEdgePartitions((_, part) => part.groupEdges(merge)) + new GraphImpl(vertices, replicatedVertexView.withEdges(newEdges)) + } + + // /////////////////////////////////////////////////////////////////////////////////////////////// + // Lower level transformation methods + // /////////////////////////////////////////////////////////////////////////////////////////////// + + override def aggregateMessagesWithActiveSet[A: ClassTag]( + sendMsg: EdgeContext[VD, ED, A] => Unit, + mergeMsg: (A, A) => A, + tripletFields: TripletFields, + activeSetOpt: Option[(VertexRDD[_], EdgeDirection)]): VertexRDD[A] = { + + vertices.cache() + // For each vertex, replicate its attribute only to partitions where it is + // in the relevant position in an edge. + replicatedVertexView.upgrade(vertices, tripletFields.useSrc, tripletFields.useDst) + val view = activeSetOpt match { + case Some((activeSet, _)) => + replicatedVertexView.withActiveSet(activeSet) + case None => + replicatedVertexView + } + val activeDirectionOpt = activeSetOpt.map(_._2) + + // Map and combine. + val preAgg = view.edges.partitionsRDD + .mapPartitions(_.flatMap { case (_, edgePartition) => + // Choose scan method + val activeFraction = + edgePartition.numActives.getOrElse(0) / edgePartition.indexSize.toFloat + activeDirectionOpt match { + case Some(EdgeDirection.Both) => + if (activeFraction < 0.8) { + edgePartition + .aggregateMessagesIndexScan(sendMsg, mergeMsg, tripletFields, EdgeActiveness.Both) + } else { + edgePartition + .aggregateMessagesEdgeScan(sendMsg, mergeMsg, tripletFields, EdgeActiveness.Both) + } + case Some(EdgeDirection.Either) => + // TODO: Because we only have a clustered index on the source vertex ID, we can't filter + // the index here. Instead we have to scan all edges and then do the filter. + edgePartition + .aggregateMessagesEdgeScan(sendMsg, mergeMsg, tripletFields, EdgeActiveness.Either) + case Some(EdgeDirection.Out) => + if (activeFraction < 0.8) { + edgePartition.aggregateMessagesIndexScan( + sendMsg, + mergeMsg, + tripletFields, + EdgeActiveness.SrcOnly) + } else { + edgePartition.aggregateMessagesEdgeScan( + sendMsg, + mergeMsg, + tripletFields, + EdgeActiveness.SrcOnly) + } + case Some(EdgeDirection.In) => + edgePartition + .aggregateMessagesEdgeScan(sendMsg, mergeMsg, tripletFields, EdgeActiveness.DstOnly) + case _ => // None + edgePartition + .aggregateMessagesEdgeScan(sendMsg, mergeMsg, tripletFields, EdgeActiveness.Neither) + } + }) + .setName("GraphImpl.aggregateMessages - preAgg") + + // do the final reduction reusing the index map + vertices.aggregateUsingIndex(preAgg, mergeMsg) + } + + override def outerJoinVertices[U: ClassTag, VD2: ClassTag](other: RDD[(VertexId, U)])( + updateF: (VertexId, VD, Option[U]) => VD2)(implicit + eq: VD =:= VD2 = null): Graph[VD2, ED] = { + // The implicit parameter eq will be populated by the compiler if VD and VD2 are equal, and left + // null if not + if (eq != null) { + vertices.cache() + // updateF preserves type, so we can use incremental replication + val newVerts = vertices.leftJoin(other)(updateF).cache() + val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) + val newReplicatedVertexView = replicatedVertexView + .asInstanceOf[ReplicatedVertexView[VD2, ED]] + .updateVertices(changedVerts) + new GraphImpl(newVerts, newReplicatedVertexView) + } else { + // updateF does not preserve type, so we must re-replicate all vertices + val newVerts = vertices.leftJoin(other)(updateF) + GraphImpl(newVerts, replicatedVertexView.edges) + } + } + +} // end of class GraphImpl + +object GraphImpl { + + /** + * Create a graph from edges, setting referenced vertices to `defaultVertexAttr`. + */ + def apply[VD: ClassTag, ED: ClassTag]( + edges: RDD[Edge[ED]], + defaultVertexAttr: VD, + edgeStorageLevel: StorageLevel, + vertexStorageLevel: StorageLevel): GraphImpl[VD, ED] = { + fromEdgeRDD(EdgeRDD.fromEdges(edges), defaultVertexAttr, edgeStorageLevel, vertexStorageLevel) + } + + /** + * Create a graph from EdgePartitions, setting referenced vertices to `defaultVertexAttr`. + */ + def fromEdgePartitions[VD: ClassTag, ED: ClassTag]( + edgePartitions: RDD[(PartitionID, EdgePartition[ED, VD])], + defaultVertexAttr: VD, + edgeStorageLevel: StorageLevel, + vertexStorageLevel: StorageLevel): GraphImpl[VD, ED] = { + fromEdgeRDD( + EdgeRDD.fromEdgePartitions(edgePartitions), + defaultVertexAttr, + edgeStorageLevel, + vertexStorageLevel) + } + + /** + * Create a graph from vertices and edges, setting missing vertices to `defaultVertexAttr`. + */ + def apply[VD: ClassTag, ED: ClassTag]( + vertices: RDD[(VertexId, VD)], + edges: RDD[Edge[ED]], + defaultVertexAttr: VD, + edgeStorageLevel: StorageLevel, + vertexStorageLevel: StorageLevel): GraphImpl[VD, ED] = { + val edgeRDD = EdgeRDD + .fromEdges(edges)(classTag[ED], classTag[VD]) + .withTargetStorageLevel(edgeStorageLevel) + val vertexRDD = VertexRDD(vertices, edgeRDD, defaultVertexAttr) + .withTargetStorageLevel(vertexStorageLevel) + GraphImpl(vertexRDD, edgeRDD) + } + + /** + * Create a graph from a VertexRDD and an EdgeRDD with arbitrary replicated vertices. The + * VertexRDD must already be set up for efficient joins with the EdgeRDD by calling + * `VertexRDD.withEdges` or an appropriate VertexRDD constructor. + */ + def apply[VD: ClassTag, ED: ClassTag]( + vertices: VertexRDD[VD], + edges: EdgeRDD[ED]): GraphImpl[VD, ED] = { + + vertices.cache() + + // Convert the vertex partitions in edges to the correct type + val newEdges = edges + .asInstanceOf[EdgeRDDImpl[ED, _]] + .mapEdgePartitions((_, part) => part.withoutVertexAttributes[VD]()) + .cache() + + GraphImpl.fromExistingRDDs(vertices, newEdges) + } + + /** + * Create a graph from a VertexRDD and an EdgeRDD with the same replicated vertex type as the + * vertices. The VertexRDD must already be set up for efficient joins with the EdgeRDD by + * calling `VertexRDD.withEdges` or an appropriate VertexRDD constructor. + */ + def fromExistingRDDs[VD: ClassTag, ED: ClassTag]( + vertices: VertexRDD[VD], + edges: EdgeRDD[ED]): GraphImpl[VD, ED] = { + new GraphImpl(vertices, new ReplicatedVertexView(edges.asInstanceOf[EdgeRDDImpl[ED, VD]])) + } + + /** + * Create a graph from an EdgeRDD with the correct vertex type, setting missing vertices to + * `defaultVertexAttr`. The vertices will have the same number of partitions as the EdgeRDD. + */ + private def fromEdgeRDD[VD: ClassTag, ED: ClassTag]( + edges: EdgeRDDImpl[ED, VD], + defaultVertexAttr: VD, + edgeStorageLevel: StorageLevel, + vertexStorageLevel: StorageLevel): GraphImpl[VD, ED] = { + val edgesCached = edges.withTargetStorageLevel(edgeStorageLevel).cache() + val vertices = + VertexRDD + .fromEdges(edgesCached, edgesCached.partitions.length, defaultVertexAttr) + .withTargetStorageLevel(vertexStorageLevel) + fromExistingRDDs(vertices, edgesCached) + } + +} // end of object GraphImpl diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/ReplicatedVertexView.scala new file mode 100644 index 000000000..bb237543b --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/ReplicatedVertexView.scala @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.impl +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.rdd.RDD + +import scala.reflect.ClassTag + +/** + * Manages shipping vertex attributes to the edge partitions of an [[graphframes.graphx.EdgeRDD]]. + * Vertex attributes may be partially shipped to construct a triplet view with vertex attributes + * on only one side, and they may be updated. An active vertex set may additionally be shipped to + * the edge partitions. Be careful not to store a reference to `edges`, since it may be modified + * when the attribute shipping level is upgraded. + */ +private[impl] class ReplicatedVertexView[VD: ClassTag, ED: ClassTag]( + var edges: EdgeRDDImpl[ED, VD], + var hasSrcId: Boolean = false, + var hasDstId: Boolean = false) { + + /** + * Return a new `ReplicatedVertexView` with the specified `EdgeRDD`, which must have the same + * shipping level. + */ + def withEdges[VD2: ClassTag, ED2: ClassTag]( + _edges: EdgeRDDImpl[ED2, VD2]): ReplicatedVertexView[VD2, ED2] = { + new ReplicatedVertexView(_edges, hasSrcId, hasDstId) + } + + /** + * Return a new `ReplicatedVertexView` where edges are reversed and shipping levels are swapped + * to match. + */ + def reverse(): ReplicatedVertexView[VD, ED] = { + val newEdges = edges.mapEdgePartitions((_, part) => part.reverse) + new ReplicatedVertexView(newEdges, hasDstId, hasSrcId) + } + + /** + * Upgrade the shipping level in-place to the specified levels by shipping vertex attributes + * from `vertices`. This operation modifies the `ReplicatedVertexView`, and callers can access + * `edges` afterwards to obtain the upgraded view. + */ + def upgrade(vertices: VertexRDD[VD], includeSrc: Boolean, includeDst: Boolean): Unit = { + val shipSrc = includeSrc && !hasSrcId + val shipDst = includeDst && !hasDstId + if (shipSrc || shipDst) { + val shippedVerts: RDD[(Int, VertexAttributeBlock[VD])] = + vertices + .shipVertexAttributes(shipSrc, shipDst) + .setName("ReplicatedVertexView.upgrade(%s, %s) - shippedVerts %s %s (broadcast)" + .format(includeSrc, includeDst, shipSrc, shipDst)) + .partitionBy(edges.partitioner.get) + val newEdges = edges.withPartitionsRDD(edges.partitionsRDD.zipPartitions(shippedVerts) { + (ePartIter, shippedVertsIter) => + ePartIter.map { case (pid, edgePartition) => + (pid, edgePartition.updateVertices(shippedVertsIter.flatMap(_._2.iterator))) + } + }) + edges = newEdges + hasSrcId = includeSrc + hasDstId = includeDst + } + } + + /** + * Return a new `ReplicatedVertexView` where the `activeSet` in each edge partition contains + * only vertex ids present in `actives`. This ships a vertex id to all edge partitions where it + * is referenced, ignoring the attribute shipping level. + */ + def withActiveSet(actives: VertexRDD[_]): ReplicatedVertexView[VD, ED] = { + val shippedActives = actives + .shipVertexIds() + .setName("ReplicatedVertexView.withActiveSet - shippedActives (broadcast)") + .partitionBy(edges.partitioner.get) + + val newEdges = edges.withPartitionsRDD(edges.partitionsRDD.zipPartitions(shippedActives) { + (ePartIter, shippedActivesIter) => + ePartIter.map { case (pid, edgePartition) => + (pid, edgePartition.withActiveSet(shippedActivesIter.flatMap(_._2.iterator))) + } + }) + new ReplicatedVertexView(newEdges, hasSrcId, hasDstId) + } + + /** + * Return a new `ReplicatedVertexView` where vertex attributes in edge partition are updated + * using `updates`. This ships a vertex attribute only to the edge partitions where it is in the + * position(s) specified by the attribute shipping level. + */ + def updateVertices(updates: VertexRDD[VD]): ReplicatedVertexView[VD, ED] = { + val shippedVerts = updates + .shipVertexAttributes(hasSrcId, hasDstId) + .setName("ReplicatedVertexView.updateVertices - shippedVerts %s %s (broadcast)" + .format(hasSrcId, hasDstId)) + .partitionBy(edges.partitioner.get) + + val newEdges = edges.withPartitionsRDD(edges.partitionsRDD.zipPartitions(shippedVerts) { + (ePartIter, shippedVertsIter) => + ePartIter.map { case (pid, edgePartition) => + (pid, edgePartition.updateVertices(shippedVertsIter.flatMap(_._2.iterator))) + } + }) + new ReplicatedVertexView(newEdges, hasSrcId, hasDstId) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/RoutingTablePartition.scala new file mode 100644 index 000000000..ac79dee73 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/RoutingTablePartition.scala @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.impl + +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.graphframes.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.BitSet +import org.apache.spark.util.collection.PrimitiveVector + +private[graphx] object RoutingTablePartition { + + /** + * A message from an edge partition to a vertex specifying the position in which the edge + * partition references the vertex (src, dst, or both). The edge partition is encoded in the + * lower 30 bits of the Int, and the position is encoded in the upper 2 bits of the Int. + */ + type RoutingTableMessage = (VertexId, Int) + + private def toMessage(vid: VertexId, pid: PartitionID, position: Byte): RoutingTableMessage = { + val positionUpper2 = position << 30 + val pidLower30 = pid & 0x3fffffff + (vid, positionUpper2 | pidLower30) + } + + private def vidFromMessage(msg: RoutingTableMessage): VertexId = msg._1 + private def pidFromMessage(msg: RoutingTableMessage): PartitionID = msg._2 & 0x3fffffff + private def positionFromMessage(msg: RoutingTableMessage): Byte = (msg._2 >> 30).toByte + + val empty: RoutingTablePartition = new RoutingTablePartition(Array.empty) + + /** Generate a `RoutingTableMessage` for each vertex referenced in `edgePartition`. */ + def edgePartitionToMsgs( + pid: PartitionID, + edgePartition: EdgePartition[_, _]): Iterator[RoutingTableMessage] = { + // Determine which positions each vertex id appears in using a map where the low 2 bits + // represent src and dst + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, Byte] + edgePartition.iterator.foreach { e => + map.changeValue(e.srcId, 0x1, (b: Byte) => (b | 0x1).toByte) + map.changeValue(e.dstId, 0x2, (b: Byte) => (b | 0x2).toByte) + } + map.iterator.map { vidAndPosition => + val vid = vidAndPosition._1 + val position = vidAndPosition._2 + toMessage(vid, pid, position) + } + } + + /** Build a `RoutingTablePartition` from `RoutingTableMessage`s. */ + def fromMsgs( + numEdgePartitions: Int, + iter: Iterator[RoutingTableMessage]): RoutingTablePartition = { + val pid2vid = Array.fill(numEdgePartitions)(new PrimitiveVector[VertexId]) + val srcFlags = Array.fill(numEdgePartitions)(new PrimitiveVector[Boolean]) + val dstFlags = Array.fill(numEdgePartitions)(new PrimitiveVector[Boolean]) + for (msg <- iter) { + val vid = vidFromMessage(msg) + val pid = pidFromMessage(msg) + val position = positionFromMessage(msg) + pid2vid(pid) += vid + srcFlags(pid) += (position & 0x1) != 0 + dstFlags(pid) += (position & 0x2) != 0 + } + + new RoutingTablePartition(pid2vid.zipWithIndex.map { case (vids, pid) => + (vids.trim().array, toBitSet(srcFlags(pid)), toBitSet(dstFlags(pid))) + }) + } + + /** Compact the given vector of Booleans into a BitSet. */ + private def toBitSet(flags: PrimitiveVector[Boolean]): BitSet = { + val bitset = new BitSet(flags.size) + var i = 0 + while (i < flags.size) { + if (flags(i)) { + bitset.set(i) + } + i += 1 + } + bitset + } +} + +/** + * Stores the locations of edge-partition join sites for each vertex attribute in a particular + * vertex partition. This provides routing information for shipping vertex attributes to edge + * partitions. + */ +private[graphx] class RoutingTablePartition( + private val routingTable: Array[(Array[VertexId], BitSet, BitSet)]) + extends Serializable { + + /** The maximum number of edge partitions this `RoutingTablePartition` is built to join with. */ + val numEdgePartitions: Int = routingTable.length + + /** Returns the number of vertices that will be sent to the specified edge partition. */ + def partitionSize(pid: PartitionID): Int = routingTable(pid)._1.length + + /** Returns an iterator over all vertex ids stored in this `RoutingTablePartition`. */ + def iterator: Iterator[VertexId] = routingTable.iterator.flatMap(_._1.iterator) + + /** Returns a new RoutingTablePartition reflecting a reversal of all edge directions. */ + def reverse: RoutingTablePartition = { + new RoutingTablePartition(routingTable.map { case (vids, srcVids, dstVids) => + (vids, dstVids, srcVids) + }) + } + + /** + * Runs `f` on each vertex id to be sent to the specified edge partition. Vertex ids can be + * filtered by the position they have in the edge partition. + */ + def foreachWithinEdgePartition(pid: PartitionID, includeSrc: Boolean, includeDst: Boolean)( + f: VertexId => Unit): Unit = { + val (vidsCandidate, srcVids, dstVids) = routingTable(pid) + vidsCandidate.length + if (includeSrc && includeDst) { + // Avoid checks for performance + vidsCandidate.iterator.foreach(f) + } else if (!includeSrc && !includeDst) { + // Do nothing + } else { + val relevantVids = if (includeSrc) srcVids else dstVids + relevantVids.iterator.foreach { i => f(vidsCandidate(i)) } + } + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/ShippableVertexPartition.scala new file mode 100644 index 000000000..a9f0fa5c7 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/ShippableVertexPartition.scala @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.impl +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.graphframes.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.BitSet +import org.apache.spark.util.collection.PrimitiveVector + +import scala.annotation.nowarn +import scala.reflect.ClassTag + +/** Stores vertex attributes to ship to an edge partition. */ +@nowarn +private[graphx] class VertexAttributeBlock[VD: ClassTag]( + val vids: Array[VertexId], + val attrs: Array[VD]) + extends Serializable { + def iterator: Iterator[(VertexId, VD)] = + vids.indices.iterator.map { i => (vids(i), attrs(i)) } +} + +private[graphx] object ShippableVertexPartition { + + /** Construct a `ShippableVertexPartition` from the given vertices without any routing table. */ + def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)]): ShippableVertexPartition[VD] = + apply(iter, RoutingTablePartition.empty, null.asInstanceOf[VD], (a, _) => a) + + /** + * Construct a `ShippableVertexPartition` from the given vertices with the specified routing + * table, filling in missing vertices mentioned in the routing table using `defaultVal`. + */ + def apply[VD: ClassTag]( + iter: Iterator[(VertexId, VD)], + routingTable: RoutingTablePartition, + defaultVal: VD): ShippableVertexPartition[VD] = + apply(iter, routingTable, defaultVal, (a, _) => a) + + /** + * Construct a `ShippableVertexPartition` from the given vertices with the specified routing + * table, filling in missing vertices mentioned in the routing table using `defaultVal`, and + * merging duplicate vertex attribute with mergeFunc. + */ + def apply[VD: ClassTag]( + iter: Iterator[(VertexId, VD)], + routingTable: RoutingTablePartition, + defaultVal: VD, + mergeFunc: (VD, VD) => VD): ShippableVertexPartition[VD] = { + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + // Merge the given vertices using mergeFunc + iter.foreach { pair => + map.setMerge(pair._1, pair._2, mergeFunc) + } + // Fill in missing vertices mentioned in the routing table + routingTable.iterator.foreach { vid => + map.changeValue(vid, defaultVal, identity) + } + + new ShippableVertexPartition(map.keySet, map._values, map.keySet.getBitSet, routingTable) + } + + /** + * Implicit conversion to allow invoking `VertexPartitionBase` operations directly on a + * `ShippableVertexPartition`. + */ + implicit def shippablePartitionToOps[VD: ClassTag]( + partition: ShippableVertexPartition[VD]): ShippableVertexPartitionOps[VD] = + new ShippableVertexPartitionOps(partition) + + /** + * Implicit evidence that `ShippableVertexPartition` is a member of the + * `VertexPartitionBaseOpsConstructor` typeclass. This enables invoking `VertexPartitionBase` + * operations on a `ShippableVertexPartition` via an evidence parameter, as in + * [[VertexPartitionBaseOps]]. + */ + implicit object ShippableVertexPartitionOpsConstructor + extends VertexPartitionBaseOpsConstructor[ShippableVertexPartition] { + def toOps[VD: ClassTag](partition: ShippableVertexPartition[VD]) + : VertexPartitionBaseOps[VD, ShippableVertexPartition] = shippablePartitionToOps( + partition) + } +} + +/** + * A map from vertex id to vertex attribute that additionally stores edge partition join sites for + * each vertex attribute, enabling joining with an + * [[org.apache.spark.graphframes.graphx.EdgeRDD]]. + */ +private[graphx] class ShippableVertexPartition[VD: ClassTag]( + val index: VertexIdToIndexMap, + val values: Array[VD], + val mask: BitSet, + val routingTable: RoutingTablePartition) + extends VertexPartitionBase[VD] { + + /** Return a new ShippableVertexPartition with the specified routing table. */ + def withRoutingTable(_routingTable: RoutingTablePartition): ShippableVertexPartition[VD] = { + new ShippableVertexPartition(index, values, mask, _routingTable) + } + + /** + * Generate a `VertexAttributeBlock` for each edge partition keyed on the edge partition ID. The + * `VertexAttributeBlock` contains the vertex attributes from the current partition that are + * referenced in the specified positions in the edge partition. + */ + def shipVertexAttributes( + shipSrc: Boolean, + shipDst: Boolean): Iterator[(PartitionID, VertexAttributeBlock[VD])] = { + Iterator.tabulate(routingTable.numEdgePartitions) { pid => + val initialSize = if (shipSrc && shipDst) routingTable.partitionSize(pid) else 64 + val vids = new PrimitiveVector[VertexId](initialSize) + val attrs = new PrimitiveVector[VD](initialSize) + routingTable.foreachWithinEdgePartition(pid, shipSrc, shipDst) { vid => + if (isDefined(vid)) { + vids += vid + attrs += this(vid) + } + } + (pid, new VertexAttributeBlock(vids.trim().array, attrs.trim().array)) + } + } + + /** + * Generate a `VertexId` array for each edge partition keyed on the edge partition ID. The array + * contains the visible vertex ids from the current partition that are referenced in the edge + * partition. + */ + def shipVertexIds(): Iterator[(PartitionID, Array[VertexId])] = { + Iterator.tabulate(routingTable.numEdgePartitions) { pid => + val vids = new PrimitiveVector[VertexId](routingTable.partitionSize(pid)) + routingTable.foreachWithinEdgePartition(pid, true, true) { vid => + if (isDefined(vid)) { + vids += vid + } + } + (pid, vids.trim().array) + } + } +} + +private[graphx] class ShippableVertexPartitionOps[VD: ClassTag]( + self: ShippableVertexPartition[VD]) + extends VertexPartitionBaseOps[VD, ShippableVertexPartition](self) { + + def withIndex(index: VertexIdToIndexMap): ShippableVertexPartition[VD] = { + new ShippableVertexPartition(index, self.values, self.mask, self.routingTable) + } + + def withValues[VD2: ClassTag](values: Array[VD2]): ShippableVertexPartition[VD2] = { + new ShippableVertexPartition(self.index, values, self.mask, self.routingTable) + } + + def withMask(mask: BitSet): ShippableVertexPartition[VD] = { + new ShippableVertexPartition(self.index, self.values, mask, self.routingTable) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/VertexPartition.scala new file mode 100644 index 000000000..80b022c4c --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/VertexPartition.scala @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.impl + +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.util.collection.BitSet + +import scala.reflect.ClassTag + +private[graphx] object VertexPartition { + + /** Construct a `VertexPartition` from the given vertices. */ + def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)]): VertexPartition[VD] = { + val (index, values, mask) = VertexPartitionBase.initFrom(iter) + new VertexPartition(index, values, mask) + } + + /** + * Implicit conversion to allow invoking `VertexPartitionBase` operations directly on a + * `VertexPartition`. + */ + implicit def partitionToOps[VD: ClassTag]( + partition: VertexPartition[VD]): VertexPartitionOps[VD] = new VertexPartitionOps(partition) + + /** + * Implicit evidence that `VertexPartition` is a member of the + * `VertexPartitionBaseOpsConstructor` typeclass. This enables invoking `VertexPartitionBase` + * operations on a `VertexPartition` via an evidence parameter, as in + * [[VertexPartitionBaseOps]]. + */ + implicit object VertexPartitionOpsConstructor + extends VertexPartitionBaseOpsConstructor[VertexPartition] { + def toOps[VD: ClassTag]( + partition: VertexPartition[VD]): VertexPartitionBaseOps[VD, VertexPartition] = + partitionToOps(partition) + } +} + +/** A map from vertex id to vertex attribute. */ +private[graphx] class VertexPartition[VD: ClassTag]( + val index: VertexIdToIndexMap, + val values: Array[VD], + val mask: BitSet) + extends VertexPartitionBase[VD] + +private[graphx] class VertexPartitionOps[VD: ClassTag](self: VertexPartition[VD]) + extends VertexPartitionBaseOps[VD, VertexPartition](self) { + + def withIndex(index: VertexIdToIndexMap): VertexPartition[VD] = { + new VertexPartition(index, self.values, self.mask) + } + + def withValues[VD2: ClassTag](values: Array[VD2]): VertexPartition[VD2] = { + new VertexPartition(self.index, values, self.mask) + } + + def withMask(mask: BitSet): VertexPartition[VD] = { + new VertexPartition(self.index, self.values, mask) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/VertexPartitionBase.scala new file mode 100644 index 000000000..83c5e48b1 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/VertexPartitionBase.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.impl + +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.graphframes.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.BitSet + +import scala.annotation.nowarn +import scala.reflect.ClassTag + +private[graphx] object VertexPartitionBase { + + /** + * Construct the constituents of a VertexPartitionBase from the given vertices, merging + * duplicate entries arbitrarily. + */ + def initFrom[VD: ClassTag]( + iter: Iterator[(VertexId, VD)]): (VertexIdToIndexMap, Array[VD], BitSet) = { + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + iter.foreach { pair => + map(pair._1) = pair._2 + } + (map.keySet, map._values, map.keySet.getBitSet) + } + + /** + * Construct the constituents of a VertexPartitionBase from the given vertices, merging + * duplicate entries using `mergeFunc`. + */ + def initFrom[VD: ClassTag]( + iter: Iterator[(VertexId, VD)], + mergeFunc: (VD, VD) => VD): (VertexIdToIndexMap, Array[VD], BitSet) = { + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + iter.foreach { pair => + map.setMerge(pair._1, pair._2, mergeFunc) + } + (map.keySet, map._values, map.keySet.getBitSet) + } +} + +/** + * An abstract map from vertex id to vertex attribute. [[VertexPartition]] is the corresponding + * concrete implementation. [[VertexPartitionBaseOps]] provides a variety of operations for + * VertexPartitionBase and subclasses that provide implicit evidence of membership in the + * `VertexPartitionBaseOpsConstructor` typeclass (for example, + * `VertexPartition.VertexPartitionOpsConstructor`). + */ +@nowarn +private[graphx] abstract class VertexPartitionBase[@specialized(Long, Int, Double) VD: ClassTag] + extends Serializable { + + def index: VertexIdToIndexMap + def values: Array[VD] + def mask: BitSet + + val capacity: Int = index.capacity + + def size: Int = mask.cardinality() + + /** Return the vertex attribute for the given vertex ID. */ + def apply(vid: VertexId): VD = values(index.getPos(vid)) + + def isDefined(vid: VertexId): Boolean = { + val pos = index.getPos(vid) + pos >= 0 && mask.get(pos) + } + + def iterator: Iterator[(VertexId, VD)] = + mask.iterator.map(ind => (index.getValue(ind), values(ind))) +} + +/** + * A typeclass for subclasses of `VertexPartitionBase` representing the ability to wrap them in a + * `VertexPartitionBaseOps`. + */ +private[graphx] trait VertexPartitionBaseOpsConstructor[T[X] <: VertexPartitionBase[X]] { + def toOps[VD: ClassTag](partition: T[VD]): VertexPartitionBaseOps[VD, T] +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/VertexPartitionBaseOps.scala new file mode 100644 index 000000000..1c9927cb8 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/VertexPartitionBaseOps.scala @@ -0,0 +1,256 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.impl + +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.graphframes.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.internal.Logging +import org.apache.spark.util.collection.BitSet + +import scala.reflect.ClassTag + +/** + * A class containing additional operations for subclasses of VertexPartitionBase that provide + * implicit evidence of membership in the `VertexPartitionBaseOpsConstructor` typeclass (for + * example, `VertexPartition.VertexPartitionOpsConstructor`). + */ +private[graphx] abstract class VertexPartitionBaseOps[ + VD: ClassTag, + Self[X] <: VertexPartitionBase[X]: VertexPartitionBaseOpsConstructor](self: Self[VD]) + extends Serializable + with Logging { + + def withIndex(index: VertexIdToIndexMap): Self[VD] + def withValues[VD2: ClassTag](values: Array[VD2]): Self[VD2] + def withMask(mask: BitSet): Self[VD] + + /** + * Pass each vertex attribute along with the vertex id through a map function and retain the + * original RDD's partitioning and index. + * + * @tparam VD2 + * the type returned by the map function + * + * @param f + * the function applied to each vertex id and vertex attribute in the RDD + * + * @return + * a new VertexPartition with values obtained by applying `f` to each of the entries in the + * original VertexRDD. The resulting VertexPartition retains the same index. + */ + def map[VD2: ClassTag](f: (VertexId, VD) => VD2): Self[VD2] = { + // Construct a view of the map transformation + val newValues = new Array[VD2](self.capacity) + var i = self.mask.nextSetBit(0) + while (i >= 0) { + newValues(i) = f(self.index.getValue(i), self.values(i)) + i = self.mask.nextSetBit(i + 1) + } + this.withValues(newValues) + } + + /** + * Restrict the vertex set to the set of vertices satisfying the given predicate. + * + * @param pred + * the user defined predicate + * + * @note + * The vertex set preserves the original index structure which means that the returned RDD can + * be easily joined with the original vertex-set. Furthermore, the filter only modifies the + * bitmap index and so no new values are allocated. + */ + def filter(pred: (VertexId, VD) => Boolean): Self[VD] = { + // Allocate the array to store the results into + val newMask = new BitSet(self.capacity) + // Iterate over the active bits in the old mask and evaluate the predicate + var i = self.mask.nextSetBit(0) + while (i >= 0) { + if (pred(self.index.getValue(i), self.values(i))) { + newMask.set(i) + } + i = self.mask.nextSetBit(i + 1) + } + this.withMask(newMask) + } + + /** Hides the VertexId's that are the same between `this` and `other`. */ + def minus(other: Self[VD]): Self[VD] = { + if (self.index != other.index) { + logWarning("Minus operations on two VertexPartitions with different indexes is slow.") + minus(createUsingIndex(other.iterator)) + } else { + self.withMask(self.mask.andNot(other.mask)) + } + } + + /** Hides the VertexId's that are the same between `this` and `other`. */ + def minus(other: Iterator[(VertexId, VD)]): Self[VD] = { + minus(createUsingIndex(other)) + } + + /** + * Hides vertices that are the same between this and other. For vertices that are different, + * keeps the values from `other`. The indices of `this` and `other` must be the same. + */ + def diff(other: Self[VD]): Self[VD] = { + if (self.index != other.index) { + logWarning("Diffing two VertexPartitions with different indexes is slow.") + diff(createUsingIndex(other.iterator)) + } else { + val newMask = self.mask & other.mask + var i = newMask.nextSetBit(0) + while (i >= 0) { + if (self.values(i) == other.values(i)) { + newMask.unset(i) + } + i = newMask.nextSetBit(i + 1) + } + this.withValues(other.values).withMask(newMask) + } + } + + /** Left outer join another VertexPartition. */ + def leftJoin[VD2: ClassTag, VD3: ClassTag](other: Self[VD2])( + f: (VertexId, VD, Option[VD2]) => VD3): Self[VD3] = { + if (self.index != other.index) { + logWarning("Joining two VertexPartitions with different indexes is slow.") + leftJoin(createUsingIndex(other.iterator))(f) + } else { + val newValues = new Array[VD3](self.capacity) + + var i = self.mask.nextSetBit(0) + while (i >= 0) { + val otherV: Option[VD2] = if (other.mask.get(i)) Some(other.values(i)) else None + newValues(i) = f(self.index.getValue(i), self.values(i), otherV) + i = self.mask.nextSetBit(i + 1) + } + this.withValues(newValues) + } + } + + /** Left outer join another iterator of messages. */ + def leftJoin[VD2: ClassTag, VD3: ClassTag](other: Iterator[(VertexId, VD2)])( + f: (VertexId, VD, Option[VD2]) => VD3): Self[VD3] = { + leftJoin(createUsingIndex(other))(f) + } + + /** Inner join another VertexPartition. */ + def innerJoin[U: ClassTag, VD2: ClassTag](other: Self[U])( + f: (VertexId, VD, U) => VD2): Self[VD2] = { + if (self.index != other.index) { + logWarning("Joining two VertexPartitions with different indexes is slow.") + innerJoin(createUsingIndex(other.iterator))(f) + } else { + val newMask = self.mask & other.mask + val newValues = new Array[VD2](self.capacity) + var i = newMask.nextSetBit(0) + while (i >= 0) { + newValues(i) = f(self.index.getValue(i), self.values(i), other.values(i)) + i = newMask.nextSetBit(i + 1) + } + this.withValues(newValues).withMask(newMask) + } + } + + /** + * Inner join an iterator of messages. + */ + def innerJoin[U: ClassTag, VD2: ClassTag](iter: Iterator[Product2[VertexId, U]])( + f: (VertexId, VD, U) => VD2): Self[VD2] = { + innerJoin(createUsingIndex(iter))(f) + } + + /** + * Similar effect as aggregateUsingIndex((a, b) => a) + */ + def createUsingIndex[VD2: ClassTag](iter: Iterator[Product2[VertexId, VD2]]): Self[VD2] = { + val newMask = new BitSet(self.capacity) + val newValues = new Array[VD2](self.capacity) + iter.foreach { pair => + val pos = self.index.getPos(pair._1) + if (pos >= 0) { + newMask.set(pos) + newValues(pos) = pair._2 + } + } + this.withValues(newValues).withMask(newMask) + } + + /** + * Similar to innerJoin, but vertices from the left side that don't appear in iter will remain + * in the partition, hidden by the bitmask. + */ + def innerJoinKeepLeft(iter: Iterator[Product2[VertexId, VD]]): Self[VD] = { + val newMask = new BitSet(self.capacity) + val newValues = new Array[VD](self.capacity) + System.arraycopy(self.values, 0, newValues, 0, newValues.length) + iter.foreach { pair => + val pos = self.index.getPos(pair._1) + if (pos >= 0) { + newMask.set(pos) + newValues(pos) = pair._2 + } + } + this.withValues(newValues).withMask(newMask) + } + + def aggregateUsingIndex[VD2: ClassTag]( + iter: Iterator[Product2[VertexId, VD2]], + reduceFunc: (VD2, VD2) => VD2): Self[VD2] = { + val newMask = new BitSet(self.capacity) + val newValues = new Array[VD2](self.capacity) + iter.foreach { product => + val vid = product._1 + val vdata = product._2 + val pos = self.index.getPos(vid) + if (pos >= 0) { + if (newMask.get(pos)) { + newValues(pos) = reduceFunc(newValues(pos), vdata) + } else { // otherwise just store the new value + newMask.set(pos) + newValues(pos) = vdata + } + } + } + this.withValues(newValues).withMask(newMask) + } + + /** + * Construct a new VertexPartition whose index contains only the vertices in the mask. + */ + def reindex(): Self[VD] = { + val hashMap = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + val arbitraryMerge = (a: VD, _: VD) => a + for ((k, v) <- self.iterator) { + hashMap.setMerge(k, v, arbitraryMerge) + } + this.withIndex(hashMap.keySet).withValues(hashMap._values).withMask(hashMap.keySet.getBitSet) + } + + /** + * Converts a vertex partition (in particular, one of type `Self`) into a + * `VertexPartitionBaseOps`. Within this class, this allows chaining the methods defined above, + * because these methods return a `Self` and this implicit conversion re-wraps that in a + * `VertexPartitionBaseOps`. This relies on the context bound on `Self`. + */ + private implicit def toOps[VD2: ClassTag]( + partition: Self[VD2]): VertexPartitionBaseOps[VD2, Self] = { + implicitly[VertexPartitionBaseOpsConstructor[Self]].toOps(partition) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/VertexRDDImpl.scala new file mode 100644 index 000000000..db188e47a --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/VertexRDDImpl.scala @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.impl + +import org.apache.spark._ +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.rdd._ +import org.apache.spark.storage.StorageLevel + +import scala.reflect.ClassTag + +class VertexRDDImpl[VD] private[graphx] ( + @transient val partitionsRDD: RDD[ShippableVertexPartition[VD]], + val targetStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY)(implicit + override protected val vdTag: ClassTag[VD]) + extends VertexRDD[VD](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { + + require(partitionsRDD.partitioner.isDefined) + + override def reindex(): VertexRDD[VD] = this.withPartitionsRDD(partitionsRDD.map(_.reindex())) + + override val partitioner = partitionsRDD.partitioner + + override protected def getPreferredLocations(s: Partition): Seq[String] = + partitionsRDD.preferredLocations(s) + + override def setName(_name: String): this.type = { + if (partitionsRDD.name != null) { + partitionsRDD.setName(partitionsRDD.name + ", " + _name) + } else { + partitionsRDD.setName(_name) + } + this + } + setName("VertexRDD") + + /** + * Persists the vertex partitions at the specified storage level, ignoring any existing target + * storage level. + */ + override def persist(newLevel: StorageLevel): this.type = { + partitionsRDD.persist(newLevel) + this + } + + override def unpersist(blocking: Boolean = false): this.type = { + partitionsRDD.unpersist(blocking) + this + } + + /** + * Persists the vertex partitions at `targetStorageLevel`, which defaults to MEMORY_ONLY. + */ + override def cache(): this.type = { + partitionsRDD.persist(targetStorageLevel) + this + } + + override def getStorageLevel: StorageLevel = partitionsRDD.getStorageLevel + + override def checkpoint(): Unit = { + partitionsRDD.checkpoint() + } + + override def isCheckpointed: Boolean = { + firstParent[ShippableVertexPartition[VD]].isCheckpointed + } + + override def getCheckpointFile: Option[String] = { + partitionsRDD.getCheckpointFile + } + + /** The number of vertices in the RDD. */ + override def count(): Long = { + partitionsRDD.map(_.size.toLong).fold(0)(_ + _) + } + + override private[graphx] def mapVertexPartitions[VD2: ClassTag]( + f: ShippableVertexPartition[VD] => ShippableVertexPartition[VD2]): VertexRDD[VD2] = { + val newPartitionsRDD = partitionsRDD.mapPartitions(_.map(f), preservesPartitioning = true) + this.withPartitionsRDD(newPartitionsRDD) + } + + override def mapValues[VD2: ClassTag](f: VD => VD2): VertexRDD[VD2] = + this.mapVertexPartitions(_.map((_, attr) => f(attr))) + + override def mapValues[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexRDD[VD2] = + this.mapVertexPartitions(_.map(f)) + + override def minus(other: RDD[(VertexId, VD)]): VertexRDD[VD] = { + minus(this.aggregateUsingIndex(other, (a: VD, _: VD) => a)) + } + + override def minus(other: VertexRDD[VD]): VertexRDD[VD] = { + other match { + case other: VertexRDD[_] if this.partitioner == other.partitioner => + this.withPartitionsRDD[VD]( + partitionsRDD.zipPartitions(other.partitionsRDD, preservesPartitioning = true) { + (thisIter, otherIter) => + val thisPart = thisIter.next() + val otherPart = otherIter.next() + Iterator(thisPart.minus(otherPart)) + }) + case _ => + this.withPartitionsRDD[VD](partitionsRDD + .zipPartitions(other.partitionBy(this.partitioner.get), preservesPartitioning = true) { + (partIter, msgs) => partIter.map(_.minus(msgs)) + }) + } + } + + override def diff(other: RDD[(VertexId, VD)]): VertexRDD[VD] = { + diff(this.aggregateUsingIndex(other, (a: VD, _: VD) => a)) + } + + override def diff(other: VertexRDD[VD]): VertexRDD[VD] = { + val otherPartition = other match { + case other: VertexRDD[_] if this.partitioner == other.partitioner => + other.partitionsRDD + case _ => + VertexRDD(other.partitionBy(this.partitioner.get)).partitionsRDD + } + val newPartitionsRDD = + partitionsRDD.zipPartitions(otherPartition, preservesPartitioning = true) { + (thisIter, otherIter) => + val thisPart = thisIter.next() + val otherPart = otherIter.next() + Iterator(thisPart.diff(otherPart)) + } + this.withPartitionsRDD(newPartitionsRDD) + } + + override def leftZipJoin[VD2: ClassTag, VD3: ClassTag](other: VertexRDD[VD2])( + f: (VertexId, VD, Option[VD2]) => VD3): VertexRDD[VD3] = { + val newPartitionsRDD = + partitionsRDD.zipPartitions(other.partitionsRDD, preservesPartitioning = true) { + (thisIter, otherIter) => + val thisPart = thisIter.next() + val otherPart = otherIter.next() + Iterator(thisPart.leftJoin(otherPart)(f)) + } + this.withPartitionsRDD(newPartitionsRDD) + } + + override def leftJoin[VD2: ClassTag, VD3: ClassTag](other: RDD[(VertexId, VD2)])( + f: (VertexId, VD, Option[VD2]) => VD3): VertexRDD[VD3] = { + // Test if the other vertex is a VertexRDD to choose the optimal join strategy. + // If the other set is a VertexRDD then we use the much more efficient leftZipJoin + other match { + case other: VertexRDD[_] if this.partitioner == other.partitioner => + leftZipJoin(other)(f) + case _ => + this.withPartitionsRDD[VD3](partitionsRDD + .zipPartitions(other.partitionBy(this.partitioner.get), preservesPartitioning = true) { + (partIter, msgs) => partIter.map(_.leftJoin(msgs)(f)) + }) + } + } + + override def innerZipJoin[U: ClassTag, VD2: ClassTag](other: VertexRDD[U])( + f: (VertexId, VD, U) => VD2): VertexRDD[VD2] = { + val newPartitionsRDD = + partitionsRDD.zipPartitions(other.partitionsRDD, preservesPartitioning = true) { + (thisIter, otherIter) => + val thisPart = thisIter.next() + val otherPart = otherIter.next() + Iterator(thisPart.innerJoin(otherPart)(f)) + } + this.withPartitionsRDD(newPartitionsRDD) + } + + override def innerJoin[U: ClassTag, VD2: ClassTag](other: RDD[(VertexId, U)])( + f: (VertexId, VD, U) => VD2): VertexRDD[VD2] = { + // Test if the other vertex is a VertexRDD to choose the optimal join strategy. + // If the other set is a VertexRDD then we use the much more efficient innerZipJoin + other match { + case other: VertexRDD[_] if this.partitioner == other.partitioner => + innerZipJoin(other)(f) + case _ => + this.withPartitionsRDD(partitionsRDD + .zipPartitions(other.partitionBy(this.partitioner.get), preservesPartitioning = true) { + (partIter, msgs) => partIter.map(_.innerJoin(msgs)(f)) + }) + } + } + + override def aggregateUsingIndex[VD2: ClassTag]( + messages: RDD[(VertexId, VD2)], + reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { + val shuffled = messages.partitionBy(this.partitioner.get) + val parts = partitionsRDD.zipPartitions(shuffled, true) { (thisIter, msgIter) => + thisIter.map(_.aggregateUsingIndex(msgIter, reduceFunc)) + } + this.withPartitionsRDD[VD2](parts) + } + + override def reverseRoutingTables(): VertexRDD[VD] = + this.mapVertexPartitions(vPart => vPart.withRoutingTable(vPart.routingTable.reverse)) + + override def withEdges(edges: EdgeRDD[_]): VertexRDD[VD] = { + val routingTables = VertexRDD.createRoutingTables(edges, this.partitioner.get) + val vertexPartitions = partitionsRDD.zipPartitions(routingTables, true) { + (partIter, routingTableIter) => + val routingTable = + if (routingTableIter.hasNext) routingTableIter.next() else RoutingTablePartition.empty + partIter.map(_.withRoutingTable(routingTable)) + } + this.withPartitionsRDD(vertexPartitions) + } + + override private[graphx] def withPartitionsRDD[VD2: ClassTag]( + partitionsRDD: RDD[ShippableVertexPartition[VD2]]): VertexRDD[VD2] = { + new VertexRDDImpl(partitionsRDD, this.targetStorageLevel) + } + + override private[graphx] def withTargetStorageLevel( + targetStorageLevel: StorageLevel): VertexRDD[VD] = { + new VertexRDDImpl(this.partitionsRDD, targetStorageLevel) + } + + override private[graphx] def shipVertexAttributes( + shipSrc: Boolean, + shipDst: Boolean): RDD[(PartitionID, VertexAttributeBlock[VD])] = { + partitionsRDD.mapPartitions(_.flatMap(_.shipVertexAttributes(shipSrc, shipDst))) + } + + override private[graphx] def shipVertexIds(): RDD[(PartitionID, Array[VertexId])] = { + partitionsRDD.mapPartitions(_.flatMap(_.shipVertexIds())) + } + +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/package.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/package.scala new file mode 100644 index 000000000..947447531 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/impl/package.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.apache.spark.util.collection.OpenHashSet + +package object impl { + private[graphx] type VertexIdToIndexMap = OpenHashSet[VertexId] +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/ConnectedComponents.scala new file mode 100644 index 000000000..21f40a197 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/ConnectedComponents.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.lib +import org.apache.spark.graphframes.graphx._ + +import scala.annotation.nowarn +import scala.reflect.ClassTag + +/** Connected components algorithm. */ +object ConnectedComponents { + + /** + * Compute the connected component membership of each vertex and return a graph with the vertex + * value containing the lowest vertex id in the connected component containing that vertex. + * + * @tparam VD + * the vertex attribute type (discarded in the computation) + * @tparam ED + * the edge attribute type (preserved in the computation) + * @param graph + * the graph for which to compute the connected components + * @param maxIterations + * the maximum number of iterations to run for + * @return + * a graph with vertex attributes containing the smallest vertex in each connected component + */ + @nowarn + def run[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED], + maxIterations: Int): Graph[VertexId, ED] = { + require( + maxIterations > 0, + "Maximum of iterations must be greater than 0," + + s" but got ${maxIterations}") + + val ccGraph = graph.mapVertices { case (vid, _) => vid } + def sendMessage(edge: EdgeTriplet[VertexId, ED]): Iterator[(VertexId, VertexId)] = { + if (edge.srcAttr < edge.dstAttr) { + Iterator((edge.dstId, edge.srcAttr)) + } else if (edge.srcAttr > edge.dstAttr) { + Iterator((edge.srcId, edge.dstAttr)) + } else { + Iterator.empty + } + } + val initialMessage = Long.MaxValue + val pregelGraph = Pregel(ccGraph, initialMessage, maxIterations, EdgeDirection.Either)( + vprog = (_, attr, msg) => math.min(attr, msg), + sendMsg = sendMessage, + mergeMsg = (a, b) => math.min(a, b)) + ccGraph.unpersist() + pregelGraph + } // end of connectedComponents + + /** + * Compute the connected component membership of each vertex and return a graph with the vertex + * value containing the lowest vertex id in the connected component containing that vertex. + * + * @tparam VD + * the vertex attribute type (discarded in the computation) + * @tparam ED + * the edge attribute type (preserved in the computation) + * @param graph + * the graph for which to compute the connected components + * @return + * a graph with vertex attributes containing the smallest vertex in each connected component + */ + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[VertexId, ED] = { + run(graph, Int.MaxValue) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/LabelPropagation.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/LabelPropagation.scala new file mode 100644 index 000000000..327fc6f64 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/LabelPropagation.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.lib +import org.apache.spark.graphframes.graphx._ + +import scala.annotation.nowarn +import scala.collection.Map +import scala.collection.mutable +import scala.reflect.ClassTag + +/** Label Propagation algorithm. */ +object LabelPropagation { + + /** + * Run static Label Propagation for detecting communities in networks. + * + * Each node in the network is initially assigned to its own community. At every superstep, + * nodes send their community affiliation to all neighbors and update their state to the mode + * community affiliation of incoming messages. + * + * LPA is a standard community detection algorithm for graphs. It is very inexpensive + * computationally, although (1) convergence is not guaranteed and (2) one can end up with + * trivial solutions (all nodes are identified into a single community). + * + * @tparam ED + * the edge attribute type (not used in the computation) + * + * @param graph + * the graph for which to compute the community affiliation + * @param maxSteps + * the number of supersteps of LPA to be performed. Because this is a static implementation, + * the algorithm will run for exactly this many supersteps. + * + * @return + * a graph with vertex attributes containing the label of community affiliation + */ + def run[VD, ED: ClassTag](graph: Graph[VD, ED], maxSteps: Int): Graph[VertexId, ED] = { + require(maxSteps > 0, s"Maximum of steps must be greater than 0, but got ${maxSteps}") + + val lpaGraph = graph.mapVertices { case (vid, _) => vid } + def sendMessage(e: EdgeTriplet[VertexId, ED]): Iterator[(VertexId, Map[VertexId, Long])] = { + Iterator((e.srcId, Map(e.dstAttr -> 1L)), (e.dstId, Map(e.srcAttr -> 1L))) + } + def mergeMessage( + count1: Map[VertexId, Long], + count2: Map[VertexId, Long]): Map[VertexId, Long] = { + // Mimics the optimization of breakOut, not present in Scala 2.13, while working in 2.12 + val map = mutable.Map[VertexId, Long]() + (count1.keySet ++ count2.keySet).foreach { i => + val count1Val = count1.getOrElse(i, 0L) + val count2Val = count2.getOrElse(i, 0L) + map.put(i, count1Val + count2Val) + } + map + } + @nowarn + def vertexProgram(vid: VertexId, attr: Long, message: Map[VertexId, Long]): VertexId = { + if (message.isEmpty) attr else message.maxBy(_._2)._1 + } + val initialMessage = Map[VertexId, Long]() + Pregel(lpaGraph, initialMessage, maxIterations = maxSteps)( + vprog = vertexProgram, + sendMsg = sendMessage, + mergeMsg = mergeMessage) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/PageRank.scala new file mode 100644 index 000000000..2b7815e37 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/PageRank.scala @@ -0,0 +1,608 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.lib + +import breeze.linalg.{Vector => BV} +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.linalg.Vectors + +import scala.annotation.nowarn +import scala.reflect.ClassTag + +/** + * PageRank algorithm implementation. There are two implementations of PageRank implemented. + * + * The first implementation uses the standalone `Graph` interface and runs PageRank for a fixed + * number of iterations: + * {{{ + * var PR = Array.fill(n)( 1.0 ) + * val oldPR = Array.fill(n)( 1.0 ) + * for( iter <- 0 until numIter ) { + * swap(oldPR, PR) + * for( i <- 0 until n ) { + * PR[i] = alpha + (1 - alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum + * } + * } + * }}} + * + * The second implementation uses the `Pregel` interface and runs PageRank until convergence: + * + * {{{ + * var PR = Array.fill(n)( 1.0 ) + * val oldPR = Array.fill(n)( 0.0 ) + * while( max(abs(PR - oldPr)) > tol ) { + * swap(oldPR, PR) + * for( i <- 0 until n if abs(PR[i] - oldPR[i]) > tol ) { + * PR[i] = alpha + (1 - \alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum + * } + * } + * }}} + * + * `alpha` is the random reset probability (typically 0.15), `inNbrs[i]` is the set of neighbors + * which link to `i` and `outDeg[j]` is the out degree of vertex `j`. + * + * @note + * This is not the "normalized" PageRank and as a consequence pages that have no inlinks will + * have a PageRank of alpha. + */ +object PageRank extends Logging { + + /** + * Run PageRank for a fixed number of iterations returning a graph with vertex attributes + * containing the PageRank and edge attributes the normalized edge weight. + * + * @tparam VD + * the original vertex attribute (not used) + * @tparam ED + * the original edge attribute (not used) + * + * @param graph + * the graph on which to compute PageRank + * @param numIter + * the number of iterations of PageRank to run + * @param resetProb + * the random reset probability (alpha) + * + * @return + * the graph containing with each vertex containing the PageRank and each edge containing the + * normalized weight. + */ + def run[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED], + numIter: Int, + resetProb: Double = 0.15): Graph[Double, Double] = { + runWithOptions(graph, numIter, resetProb, None) + } + + /** + * Run an update pass of PageRank algorithm. Update the values of every node in the pageRank + * + * @param rankGraph + * the current PageRank + * @param personalized + * True if personalized pageRank + * @param resetProb + * the random reset probability (alpha) + * @param src + * the source vertex for a Personalized Page Rank + * + * @return + * the graph containing with each vertex containing the PageRank and each edge containing the + * normalized weight after a single update step. + */ + private def runUpdate( + rankGraph: Graph[Double, Double], + personalized: Boolean, + resetProb: Double, + src: VertexId): Graph[Double, Double] = { + + def delta(u: VertexId, v: VertexId): Double = { if (u == v) 1.0 else 0.0 } + // Compute the outgoing rank contributions of each vertex, perform local preaggregation, and + // do the final aggregation at the receiving vertices. Requires a shuffle for aggregation. + val rankUpdates = rankGraph.aggregateMessages[Double]( + ctx => ctx.sendToDst(ctx.srcAttr * ctx.attr), + _ + _, + TripletFields.Src) + + // Apply the final rank updates to get the new ranks, using join to preserve ranks of vertices + // that didn't receive a message. Requires a shuffle for broadcasting updated ranks to the + // edge partitions. + val rPrb = if (personalized) { (src: VertexId, id: VertexId) => + resetProb * delta(src, id) + } else { (_: VertexId, _: VertexId) => + resetProb + } + + rankGraph.outerJoinVertices(rankUpdates) { (id, _, msgSumOpt) => + rPrb(src, id) + (1.0 - resetProb) * msgSumOpt.getOrElse(0.0) + } + } + + /** + * Run PageRank for a fixed number of iterations returning a graph with vertex attributes + * containing the PageRank and edge attributes the normalized edge weight. + * + * @tparam VD + * the original vertex attribute (not used) + * @tparam ED + * the original edge attribute (not used) + * + * @param graph + * the graph on which to compute PageRank + * @param numIter + * the number of iterations of PageRank to run + * @param resetProb + * the random reset probability (alpha) + * @param srcId + * the source vertex for a Personalized Page Rank (optional) + * + * @return + * the graph containing with each vertex containing the PageRank and each edge containing the + * normalized weight. + */ + def runWithOptions[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED], + numIter: Int, + resetProb: Double = 0.15, + srcId: Option[VertexId] = None): Graph[Double, Double] = { + runWithOptions(graph, numIter, resetProb, srcId, normalized = true) + } + + /** + * Run PageRank for a fixed number of iterations returning a graph with vertex attributes + * containing the PageRank and edge attributes the normalized edge weight. + * + * @tparam VD + * the original vertex attribute (not used) + * @tparam ED + * the original edge attribute (not used) + * + * @param graph + * the graph on which to compute PageRank + * @param numIter + * the number of iterations of PageRank to run + * @param resetProb + * the random reset probability (alpha) + * @param srcId + * the source vertex for a Personalized Page Rank (optional) + * @param normalized + * whether or not to normalize rank sum + * + * @return + * the graph containing with each vertex containing the PageRank and each edge containing the + * normalized weight. + * + * @since 3.2.0 + */ + def runWithOptions[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED], + numIter: Int, + resetProb: Double, + srcId: Option[VertexId], + normalized: Boolean): Graph[Double, Double] = { + require( + numIter > 0, + "Number of iterations must be greater than 0," + + s" but got ${numIter}") + require( + resetProb >= 0 && resetProb <= 1, + "Random reset probability must belong" + + s" to [0, 1], but got ${resetProb}") + + val personalized = srcId.isDefined + val src: VertexId = srcId.getOrElse(-1L) + + // Initialize the PageRank graph with each edge attribute having + // weight 1/outDegree and each vertex with attribute 1.0. + // When running personalized pagerank, only the source vertex + // has an attribute 1.0. All others are set to 0. + var rankGraph: Graph[Double, Double] = graph + // Associate the degree with each vertex + .outerJoinVertices(graph.outDegrees) { (_, _, deg) => deg.getOrElse(0) } + // Set the weight on the edges based on the degree + .mapTriplets(e => 1.0 / e.srcAttr, TripletFields.Src) + // Set the vertex attributes to the initial pagerank values + .mapVertices { (id, _) => + if (!(id != src && personalized)) 1.0 else 0.0 + } + + var iteration = 0 + var prevRankGraph: Graph[Double, Double] = null + while (iteration < numIter) { + rankGraph.cache() + prevRankGraph = rankGraph + + rankGraph = runUpdate(rankGraph, personalized, resetProb, src) + rankGraph.cache() + rankGraph.edges.foreachPartition(_ => {}) // also materializes rankGraph.vertices + prevRankGraph.vertices.unpersist() + prevRankGraph.edges.unpersist() + iteration += 1 + } + + if (normalized) { + // SPARK-18847 If the graph has sinks (vertices with no outgoing edges), + // correct the sum of ranks + normalizeRankSum(rankGraph, personalized) + } else { + rankGraph + } + } + + /** + * Run PageRank for a fixed number of iterations returning a graph with vertex attributes + * containing the PageRank and edge attributes the normalized edge weight. + * + * @tparam VD + * the original vertex attribute (not used) + * @tparam ED + * the original edge attribute (not used) + * + * @param graph + * the graph on which to compute PageRank + * @param numIter + * the number of iterations of PageRank to run + * @param resetProb + * the random reset probability (alpha) + * @param srcId + * the source vertex for a Personalized Page Rank (optional) + * @param preRankGraph + * PageRank graph from which to keep iterating + * + * @return + * the graph containing with each vertex containing the PageRank and each edge containing the + * normalized weight. + */ + def runWithOptionsWithPreviousPageRank[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED], + numIter: Int, + resetProb: Double, + srcId: Option[VertexId], + preRankGraph: Graph[Double, Double]): Graph[Double, Double] = { + runWithOptionsWithPreviousPageRank( + graph, + numIter, + resetProb, + srcId, + normalized = true, + preRankGraph) + } + + /** + * Run PageRank for a fixed number of iterations returning a graph with vertex attributes + * containing the PageRank and edge attributes the normalized edge weight. + * + * @tparam VD + * the original vertex attribute (not used) + * @tparam ED + * the original edge attribute (not used) + * + * @param graph + * the graph on which to compute PageRank + * @param numIter + * the number of iterations of PageRank to run + * @param resetProb + * the random reset probability (alpha) + * @param srcId + * the source vertex for a Personalized Page Rank (optional) + * @param normalized + * whether or not to normalize rank sum + * @param preRankGraph + * PageRank graph from which to keep iterating + * + * @return + * the graph containing with each vertex containing the PageRank and each edge containing the + * normalized weight. + * + * @since 3.2.0 + */ + def runWithOptionsWithPreviousPageRank[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED], + numIter: Int, + resetProb: Double, + srcId: Option[VertexId], + normalized: Boolean, + preRankGraph: Graph[Double, Double]): Graph[Double, Double] = { + require( + numIter > 0, + "Number of iterations must be greater than 0," + + s" but got ${numIter}") + require( + resetProb >= 0 && resetProb <= 1, + "Random reset probability must belong" + + s" to [0, 1], but got ${resetProb}") + val graphVertices = graph.numVertices + val prePageRankVertices = preRankGraph.numVertices + require( + graphVertices == prePageRankVertices, + "Graph and previous pageRankGraph" + + s" must have the same number of vertices but got ${graphVertices} and ${prePageRankVertices}") + + val personalized = srcId.isDefined + val src: VertexId = srcId.getOrElse(-1L) + + // Initialize the PageRank graph with each edge attribute having + // weight 1/outDegree and each vertex with attribute 1.0. + // When running personalized pagerank, only the source vertex + // has an attribute 1.0. All others are set to 0. + var rankGraph: Graph[Double, Double] = preRankGraph + + var iteration = 0 + var prevRankGraph: Graph[Double, Double] = null + + while (iteration < numIter) { + rankGraph.cache() + prevRankGraph = rankGraph + + rankGraph = runUpdate(rankGraph, personalized, resetProb, src) + rankGraph.cache() + rankGraph.edges.foreachPartition(_ => {}) // also materializes rankGraph.vertices + prevRankGraph.vertices.unpersist() + prevRankGraph.edges.unpersist() + iteration += 1 + } + + if (normalized) { + // SPARK-18847 If the graph has sinks (vertices with no outgoing edges), + // correct the sum of ranks + normalizeRankSum(rankGraph, personalized) + } else { + rankGraph + } + } + + /** + * Run Personalized PageRank for a fixed number of iterations, for a set of starting nodes in + * parallel. Returns a graph with vertex attributes containing the pagerank relative to all + * starting nodes (as a sparse vector) and edge attributes the normalized edge weight + * + * @tparam VD + * The original vertex attribute (not used) + * @tparam ED + * The original edge attribute (not used) + * + * @param graph + * The graph on which to compute personalized pagerank + * @param numIter + * The number of iterations to run + * @param resetProb + * The random reset probability + * @param sources + * The list of sources to compute personalized pagerank from + * @return + * the graph with vertex attributes containing the pagerank relative to all starting nodes (as + * a sparse vector indexed by the position of nodes in the sources list) and edge attributes + * the normalized edge weight + */ + def runParallelPersonalizedPageRank[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED], + numIter: Int, + resetProb: Double = 0.15, + sources: Array[VertexId]): Graph[Vector, Double] = { + require( + numIter > 0, + "Number of iterations must be greater than 0," + + s" but got ${numIter}") + require( + resetProb >= 0 && resetProb <= 1, + "Random reset probability must belong" + + s" to [0, 1], but got ${resetProb}") + require( + sources.nonEmpty, + "The list of sources must be non-empty," + + s" but got ${sources.mkString("[", ",", "]")}") + + val zero = Vectors.sparse(sources.length, List()).asBreeze + // map of vid -> vector where for each vid, the _position of vid in source_ is set to 1.0 + val sourcesInitMap = sources.zipWithIndex.map { case (vid, i) => + val v = Vectors.sparse(sources.length, Array(i), Array(1.0)).asBreeze + (vid, v) + }.toMap + + val sc = graph.vertices.sparkContext + val sourcesInitMapBC = sc.broadcast(sourcesInitMap) + // Initialize the PageRank graph with each edge attribute having + // weight 1/outDegree and each source vertex with attribute 1.0. + var rankGraph = graph + // Associate the degree with each vertex + .outerJoinVertices(graph.outDegrees) { (_, _, deg) => deg.getOrElse(0) } + // Set the weight on the edges based on the degree + .mapTriplets(e => 1.0 / e.srcAttr, TripletFields.Src) + .mapVertices((vid, _) => sourcesInitMapBC.value.getOrElse(vid, zero)) + + var i = 0 + while (i < numIter) { + val prevRankGraph = rankGraph + // Propagates the message along outbound edges + // and adding start nodes back in with activation resetProb + val rankUpdates = rankGraph.aggregateMessages[BV[Double]]( + ctx => ctx.sendToDst(ctx.srcAttr *:* ctx.attr), + (a: BV[Double], b: BV[Double]) => a +:+ b, + TripletFields.Src) + + rankGraph = rankGraph + .outerJoinVertices(rankUpdates) { (vid, _, msgSumOpt) => + val popActivations: BV[Double] = msgSumOpt.getOrElse(zero) *:* (1.0 - resetProb) + val resetActivations = if (sourcesInitMapBC.value contains vid) { + sourcesInitMapBC.value(vid) *:* resetProb + } else { + zero + } + popActivations +:+ resetActivations + } + .cache() + + rankGraph.edges.foreachPartition(_ => {}) // also materializes rankGraph.vertices + prevRankGraph.vertices.unpersist() + prevRankGraph.edges.unpersist() + + i += 1 + } + + // SPARK-18847 If the graph has sinks (vertices with no outgoing edges) correct the sum of ranks + val rankSums = rankGraph.vertices.values.fold(zero)(_ +:+ _) + rankGraph.mapVertices { (_, attr) => + Vectors.fromBreeze(attr /:/ rankSums) + } + } + + /** + * Run a dynamic version of PageRank returning a graph with vertex attributes containing the + * PageRank and edge attributes containing the normalized edge weight. + * + * @tparam VD + * the original vertex attribute (not used) + * @tparam ED + * the original edge attribute (not used) + * + * @param graph + * the graph on which to compute PageRank + * @param tol + * the tolerance allowed at convergence (smaller => more accurate). + * @param resetProb + * the random reset probability (alpha) + * + * @return + * the graph containing with each vertex containing the PageRank and each edge containing the + * normalized weight. + */ + def runUntilConvergence[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED], + tol: Double, + resetProb: Double = 0.15): Graph[Double, Double] = { + runUntilConvergenceWithOptions(graph, tol, resetProb) + } + + /** + * Run a dynamic version of PageRank returning a graph with vertex attributes containing the + * PageRank and edge attributes containing the normalized edge weight. + * + * @tparam VD + * the original vertex attribute (not used) + * @tparam ED + * the original edge attribute (not used) + * + * @param graph + * the graph on which to compute PageRank + * @param tol + * the tolerance allowed at convergence (smaller => more accurate). + * @param resetProb + * the random reset probability (alpha) + * @param srcId + * the source vertex for a Personalized Page Rank (optional) + * + * @return + * the graph containing with each vertex containing the PageRank and each edge containing the + * normalized weight. + */ + def runUntilConvergenceWithOptions[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED], + tol: Double, + resetProb: Double = 0.15, + srcId: Option[VertexId] = None): Graph[Double, Double] = { + require(tol >= 0, s"Tolerance must be no less than 0, but got ${tol}") + require( + resetProb >= 0 && resetProb <= 1, + "Random reset probability must belong" + + s" to [0, 1], but got ${resetProb}") + + val personalized = srcId.isDefined + val src: VertexId = srcId.getOrElse(-1L) + + // Initialize the pagerankGraph with each edge attribute + // having weight 1/outDegree and each vertex with attribute 0. + val pagerankGraph: Graph[(Double, Double), Double] = graph + // Associate the degree with each vertex + .outerJoinVertices(graph.outDegrees) { (_, _, deg) => + deg.getOrElse(0) + } + // Set the weight on the edges based on the degree + .mapTriplets(e => 1.0 / e.srcAttr) + // Set the vertex attributes to (initialPR, delta = 0) + .mapVertices { (id, _) => + if (id == src) (0.0, Double.NegativeInfinity) else (0.0, 0.0) + } + .cache() + + // Define the three functions needed to implement PageRank in the GraphX + // version of Pregel + @nowarn + def vertexProgram(id: VertexId, attr: (Double, Double), msgSum: Double): (Double, Double) = { + val (oldPR, _) = attr + val newPR = oldPR + (1.0 - resetProb) * msgSum + (newPR, newPR - oldPR) + } + + @nowarn + def personalizedVertexProgram( + id: VertexId, + attr: (Double, Double), + msgSum: Double): (Double, Double) = { + val (oldPR, lastDelta) = attr + val newPR = if (lastDelta == Double.NegativeInfinity) { + 1.0 + } else { + oldPR + (1.0 - resetProb) * msgSum + } + (newPR, newPR - oldPR) + } + + def sendMessage(edge: EdgeTriplet[(Double, Double), Double]) = { + if (edge.srcAttr._2 > tol) { + Iterator((edge.dstId, edge.srcAttr._2 * edge.attr)) + } else { + Iterator.empty + } + } + + def messageCombiner(a: Double, b: Double): Double = a + b + + // The initial message received by all vertices in PageRank + val initialMessage = if (personalized) 0.0 else resetProb / (1.0 - resetProb) + + // Execute a dynamic version of Pregel. + val vp = if (personalized) { (id: VertexId, attr: (Double, Double), msgSum: Double) => + personalizedVertexProgram(id, attr, msgSum) + } else { (id: VertexId, attr: (Double, Double), msgSum: Double) => + vertexProgram(id, attr, msgSum) + } + + val rankGraph = Pregel(pagerankGraph, initialMessage, activeDirection = EdgeDirection.Out)( + vp, + sendMessage, + messageCombiner) + .mapVertices((_, attr) => attr._1) + + // SPARK-18847 If the graph has sinks (vertices with no outgoing edges) correct the sum of ranks + normalizeRankSum(rankGraph, personalized) + } + + // Normalizes the sum of ranks to n (or 1 if personalized) + private def normalizeRankSum(rankGraph: Graph[Double, Double], personalized: Boolean) = { + val rankSum = rankGraph.vertices.values.sum() + if (personalized) { + rankGraph.mapVertices((_, rank) => rank / rankSum) + } else { + val numVertices = rankGraph.numVertices + val correctionFactor = numVertices.toDouble / rankSum + rankGraph.mapVertices((_, rank) => rank * correctionFactor) + } + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/SVDPlusPlus.scala new file mode 100644 index 000000000..e7a4979ad --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/SVDPlusPlus.scala @@ -0,0 +1,237 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.lib + +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.ml.linalg.BLAS +import org.apache.spark.rdd._ + +import scala.util.Random + +/** Implementation of SVD++ algorithm. */ +object SVDPlusPlus { + + /** Configuration parameters for SVDPlusPlus. */ + class Conf( + var rank: Int, + var maxIters: Int, + var minVal: Double, + var maxVal: Double, + var gamma1: Double, + var gamma2: Double, + var gamma6: Double, + var gamma7: Double) + extends Serializable + + // scalastyle:off line.size.limit + /** + * Implement SVD++ based on "Factorization Meets the Neighborhood: a Multifaceted Collaborative + * Filtering Model", + * available here. + * + * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^^-0.5^^*sum(y)), see the details + * on page 6. + * + * @param edges + * edges for constructing the graph + * + * @param conf + * SVDPlusPlus parameters + * + * @return + * a graph with vertex attributes containing the trained model + */ + // scalastyle:on line.size.limit + def run( + edges: RDD[Edge[Double]], + conf: Conf): (Graph[(Array[Double], Array[Double], Double, Double), Double], Double) = { + require( + conf.maxIters > 0, + "Maximum of iterations must be greater than 0," + + s" but got ${conf.maxIters}") + require( + conf.maxVal > conf.minVal, + "MaxVal must be greater than MinVal," + + s" but got {maxVal: ${conf.maxVal}, minVal: ${conf.minVal}}") + + // Generate default vertex attribute + def defaultF(rank: Int): (Array[Double], Array[Double], Double, Double) = { + // TODO: use a fixed random seed + val v1 = Array.fill(rank)(Random.nextDouble()) + val v2 = Array.fill(rank)(Random.nextDouble()) + (v1, v2, 0.0, 0.0) + } + + // calculate global rating mean + edges.cache() + val (rs, rc) = edges.map(e => (e.attr, 1L)).fold((0, 0))((a, b) => (a._1 + b._1, a._2 + b._2)) + val u = rs / rc + + // construct graph + var g = Graph.fromEdges(edges, defaultF(conf.rank)).cache() + materialize(g) + edges.unpersist() + + // Calculate initial bias and norm + val t0 = g.aggregateMessages[(Long, Double)]( + ctx => { ctx.sendToSrc((1L, ctx.attr)); ctx.sendToDst((1L, ctx.attr)) }, + (g1, g2) => (g1._1 + g2._1, g1._2 + g2._2)) + + val gJoinT0 = g + .outerJoinVertices(t0) { + ( + _: VertexId, + vd: (Array[Double], Array[Double], Double, Double), + msg: Option[(Long, Double)]) => + (vd._1, vd._2, msg.get._2 / msg.get._1 - u, 1.0 / scala.math.sqrt(msg.get._1.toDouble)) + } + .cache() + materialize(gJoinT0) + g.unpersist() + g = gJoinT0 + + def sendMsgTrainF(conf: Conf, u: Double)( + ctx: EdgeContext[ + (Array[Double], Array[Double], Double, Double), + Double, + (Array[Double], Array[Double], Double)]): Unit = { + val (usr, itm) = (ctx.srcAttr, ctx.dstAttr) + val (p, q) = (usr._1, itm._1) + val rank = p.length + var pred = u + usr._3 + itm._3 + BLAS.nativeBLAS.ddot(rank, q, 1, usr._2, 1) + pred = math.max(pred, conf.minVal) + pred = math.min(pred, conf.maxVal) + val err = ctx.attr - pred + // updateP = (err * q - conf.gamma7 * p) * conf.gamma2 + val updateP = q.clone() + BLAS.nativeBLAS.dscal(rank, err * conf.gamma2, updateP, 1) + BLAS.nativeBLAS.daxpy(rank, -conf.gamma7 * conf.gamma2, p, 1, updateP, 1) + // updateQ = (err * usr._2 - conf.gamma7 * q) * conf.gamma2 + val updateQ = usr._2.clone() + BLAS.nativeBLAS.dscal(rank, err * conf.gamma2, updateQ, 1) + BLAS.nativeBLAS.daxpy(rank, -conf.gamma7 * conf.gamma2, q, 1, updateQ, 1) + // updateY = (err * usr._4 * q - conf.gamma7 * itm._2) * conf.gamma2 + val updateY = q.clone() + BLAS.nativeBLAS.dscal(rank, err * usr._4 * conf.gamma2, updateY, 1) + BLAS.nativeBLAS.daxpy(rank, -conf.gamma7 * conf.gamma2, itm._2, 1, updateY, 1) + ctx.sendToSrc((updateP, updateY, (err - conf.gamma6 * usr._3) * conf.gamma1)) + ctx.sendToDst((updateQ, updateY, (err - conf.gamma6 * itm._3) * conf.gamma1)) + } + + for (_ <- 0 until conf.maxIters) { + // Phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes + g.cache() + val t1 = g.aggregateMessages[Array[Double]]( + ctx => ctx.sendToSrc(ctx.dstAttr._2), + (g1, g2) => { + val out = g1.clone() + BLAS.nativeBLAS.daxpy(out.length, 1.0, g2, 1, out, 1) + out + }) + val gJoinT1 = g + .outerJoinVertices(t1) { + ( + _: VertexId, + vd: (Array[Double], Array[Double], Double, Double), + msg: Option[Array[Double]]) => + if (msg.isDefined) { + val out = vd._1.clone() + BLAS.nativeBLAS.daxpy(out.length, vd._4, msg.get, 1, out, 1) + (vd._1, out, vd._3, vd._4) + } else { + vd + } + } + .cache() + materialize(gJoinT1) + g.unpersist() + g = gJoinT1 + + // Phase 2, update p for user nodes and q, y for item nodes + g.cache() + val t2 = g.aggregateMessages( + sendMsgTrainF(conf, u), + ( + g1: (Array[Double], Array[Double], Double), + g2: (Array[Double], Array[Double], Double)) => { + val out1 = g1._1.clone() + BLAS.nativeBLAS.daxpy(out1.length, 1.0, g2._1, 1, out1, 1) + val out2 = g2._2.clone() + BLAS.nativeBLAS.daxpy(out2.length, 1.0, g2._2, 1, out2, 1) + (out1, out2, g1._3 + g2._3) + }) + val gJoinT2 = g + .outerJoinVertices(t2) { + ( + _: VertexId, + vd: (Array[Double], Array[Double], Double, Double), + msg: Option[(Array[Double], Array[Double], Double)]) => + { + val out1 = vd._1.clone() + BLAS.nativeBLAS.daxpy(out1.length, 1.0, msg.get._1, 1, out1, 1) + val out2 = vd._2.clone() + BLAS.nativeBLAS.daxpy(out2.length, 1.0, msg.get._2, 1, out2, 1) + (out1, out2, vd._3 + msg.get._3, vd._4) + } + } + .cache() + materialize(gJoinT2) + g.unpersist() + g = gJoinT2 + } + + // calculate error on training set + def sendMsgTestF(conf: Conf, u: Double)( + ctx: EdgeContext[(Array[Double], Array[Double], Double, Double), Double, Double]) + : Unit = { + val (usr, itm) = (ctx.srcAttr, ctx.dstAttr) + val q = itm._1 + var pred = u + usr._3 + itm._3 + BLAS.nativeBLAS.ddot(q.length, q, 1, usr._2, 1) + pred = math.max(pred, conf.minVal) + pred = math.min(pred, conf.maxVal) + val err = (ctx.attr - pred) * (ctx.attr - pred) + ctx.sendToDst(err) + } + + g.cache() + val t3 = g.aggregateMessages[Double](sendMsgTestF(conf, u), _ + _) + val gJoinT3 = g + .outerJoinVertices(t3) { + (_: VertexId, vd: (Array[Double], Array[Double], Double, Double), msg: Option[Double]) => + if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd + } + .cache() + materialize(gJoinT3) + g.unpersist() + g = gJoinT3 + + // Convert DoubleMatrix to Array[Double]: + val newVertices = g.vertices.mapValues(v => (v._1, v._2, v._3, v._4)) + (Graph[(Array[Double], Array[Double], Double, Double), Double](newVertices, g.edges), u) + } + + /** + * Forces materialization of a Graph by count()ing its RDDs. + */ + private def materialize(g: Graph[_, _]): Unit = { + val _ = (g.vertices.count(), g.edges.count()) + () + } + +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/ShortestPaths.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/ShortestPaths.scala new file mode 100644 index 000000000..58d61be9a --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/ShortestPaths.scala @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.lib + +import org.apache.spark.graphframes.graphx._ + +import scala.annotation.nowarn +import scala.collection.Map +import scala.collection.mutable +import scala.reflect.ClassTag + +/** + * Computes shortest paths to the given set of landmark vertices, returning a graph where each + * vertex attribute is a map containing the shortest-path distance to each reachable landmark. + */ +object ShortestPaths extends Serializable { + + /** Stores a map from the vertex id of a landmark to the distance to that landmark. */ + type SPMap = Map[VertexId, Int] + + private def makeMap(x: (VertexId, Int)*) = Map(x: _*) + + private def incrementMap(spmap: SPMap): SPMap = spmap.map { case (v, d) => v -> (d + 1) } + + private def addMaps(spmap1: SPMap, spmap2: SPMap): SPMap = { + // Mimics the optimization of breakOut, not present in Scala 2.13, while working in 2.12 + val map = mutable.Map[VertexId, Int]() + (spmap1.keySet ++ spmap2.keySet).foreach { k => + map.put(k, math.min(spmap1.getOrElse(k, Int.MaxValue), spmap2.getOrElse(k, Int.MaxValue))) + } + map + } + + /** + * Computes shortest paths to the given set of landmark vertices. + * + * @tparam ED + * the edge attribute type (not used in the computation) + * + * @param graph + * the graph for which to compute the shortest paths + * @param landmarks + * the list of landmark vertex ids. Shortest paths will be computed to each landmark. + * + * @return + * a graph where each vertex attribute is a map containing the shortest-path distance to each + * reachable landmark vertex. + */ + def run[VD, ED: ClassTag](graph: Graph[VD, ED], landmarks: Seq[VertexId]): Graph[SPMap, ED] = { + val spGraph = graph.mapVertices { (vid, _) => + if (landmarks.contains(vid)) makeMap(vid -> 0) else makeMap() + } + + val initialMessage = makeMap() + + @nowarn + def vertexProgram(id: VertexId, attr: SPMap, msg: SPMap): SPMap = { + addMaps(attr, msg) + } + + def sendMessage(edge: EdgeTriplet[SPMap, _]): Iterator[(VertexId, SPMap)] = { + val newAttr = incrementMap(edge.dstAttr) + if (edge.srcAttr != addMaps(newAttr, edge.srcAttr)) Iterator((edge.srcId, newAttr)) + else Iterator.empty + } + + Pregel(spGraph, initialMessage)(vertexProgram, sendMessage, addMaps) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/StronglyConnectedComponents.scala new file mode 100755 index 000000000..227392c9b --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/StronglyConnectedComponents.scala @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.lib + +import org.apache.spark.graphframes.graphx._ + +import scala.annotation.nowarn +import scala.reflect.ClassTag + +/** Strongly connected components algorithm implementation. */ +object StronglyConnectedComponents { + + /** + * Compute the strongly connected component (SCC) of each vertex and return a graph with the + * vertex value containing the lowest vertex id in the SCC containing that vertex. + * + * @tparam VD + * the vertex attribute type (discarded in the computation) + * @tparam ED + * the edge attribute type (preserved in the computation) + * + * @param graph + * the graph for which to compute the SCC + * + * @return + * a graph with vertex attributes containing the smallest vertex id in each SCC + */ + @nowarn + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], numIter: Int): Graph[VertexId, ED] = { + require( + numIter > 0, + "Number of iterations must be greater than 0," + + s" but got ${numIter}") + + // the graph we update with final SCC ids, and the graph we return at the end + var sccGraph = graph.mapVertices { case (vid, _) => vid } + // graph we are going to work with in our iterations + var sccWorkGraph = graph.mapVertices { case (vid, _) => (vid, false) }.cache() + + // helper variables to unpersist cached graphs + var prevSccGraph = sccGraph + + var numVertices = sccWorkGraph.numVertices + var iter = 0 + while (sccWorkGraph.numVertices > 0 && iter < numIter) { + iter += 1 + do { + numVertices = sccWorkGraph.numVertices + sccWorkGraph = sccWorkGraph + .outerJoinVertices(sccWorkGraph.outDegrees) { (vid, data, degreeOpt) => + if (degreeOpt.isDefined) data else (vid, true) + } + .outerJoinVertices(sccWorkGraph.inDegrees) { (vid, data, degreeOpt) => + if (degreeOpt.isDefined) data else (vid, true) + } + .cache() + + // get all vertices to be removed + val finalVertices = sccWorkGraph.vertices + .filter { case (_, (_, isFinal)) => isFinal } + .mapValues { (_, data) => data._1 } + + // write values to sccGraph + sccGraph = sccGraph + .outerJoinVertices(finalVertices) { (_, scc, opt) => + opt.getOrElse(scc) + } + .cache() + // materialize vertices and edges + sccGraph.vertices.count() + sccGraph.edges.count() + // sccGraph materialized so, unpersist can be done on previous + prevSccGraph.unpersist() + prevSccGraph = sccGraph + + // only keep vertices that are not final + sccWorkGraph = sccWorkGraph.subgraph(vpred = (_, data) => !data._2).cache() + } while (sccWorkGraph.numVertices < numVertices) + + // if iter < numIter at this point sccGraph that is returned + // will not be recomputed and pregel executions are pointless + if (iter < numIter) { + sccWorkGraph = sccWorkGraph.mapVertices { case (vid, (_, isFinal)) => (vid, isFinal) } + + // collect min of all my neighbor's scc values, update if it's smaller than mine + // then notify any neighbors with scc values larger than mine + sccWorkGraph = Pregel[(VertexId, Boolean), ED, VertexId]( + sccWorkGraph, + Long.MaxValue, + activeDirection = EdgeDirection.Out)( + (_, myScc, neighborScc) => (math.min(myScc._1, neighborScc), myScc._2), + e => { + if (e.srcAttr._1 < e.dstAttr._1) { + Iterator((e.dstId, e.srcAttr._1)) + } else { + Iterator() + } + }, + (vid1, vid2) => math.min(vid1, vid2)) + + // start at root of SCCs. Traverse values in reverse, notify all my neighbors + // do not propagate if colors do not match! + sccWorkGraph = Pregel[(VertexId, Boolean), ED, Boolean]( + sccWorkGraph, + false, + activeDirection = EdgeDirection.In)( + // vertex is final if it is the root of a color + // or it has the same color as a neighbor that is final + (vid, myScc, existsSameColorFinalNeighbor) => { + val isColorRoot = vid == myScc._1 + (myScc._1, myScc._2 || isColorRoot || existsSameColorFinalNeighbor) + }, + // activate neighbor if they are not final, you are, and you have the same color + e => { + val sameColor = e.dstAttr._1 == e.srcAttr._1 + val onlyDstIsFinal = e.dstAttr._2 && !e.srcAttr._2 + if (sameColor && onlyDstIsFinal) { + Iterator((e.srcId, e.dstAttr._2)) + } else { + Iterator() + } + }, + (final1, final2) => final1 || final2) + } + } + sccGraph + } + +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/TriangleCount.scala new file mode 100644 index 000000000..5ff98bfe6 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/lib/TriangleCount.scala @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.lib + +import org.apache.spark.graphframes.graphx._ + +import scala.annotation.nowarn +import scala.reflect.ClassTag + +/** + * Compute the number of triangles passing through each vertex. + * + * The algorithm is relatively straightforward and can be computed in three steps: + * + *
+ * + * dst -> + * (x,y) *************** _ + * | | | | + * | a | b | | + * src | | | | + * | *************** | T + * \|/ | | | | + * | c | d | | + * | | | | + * *************** - + *+ * + * where this represents the subquadrant of the adj matrix currently being + * subdivided. (x,y) represent the upper left hand corner of the subquadrant, + * and T represents the side length (guaranteed to be a power of 2). + * + * After choosing the next level subquadrant, we get the resulting sets + * of parameters: + * {{{ + * quad = a, x'=x, y'=y, T'=T/2 + * quad = b, x'=x+T/2, y'=y, T'=T/2 + * quad = c, x'=x, y'=y+T/2, T'=T/2 + * quad = d, x'=x+T/2, y'=y+T/2, T'=T/2 + * }}} + */ + @tailrec + private def chooseCell(x: Int, y: Int, t: Int): (Int, Int) = { + if (t <= 1) { + (x, y) + } else { + val newT = math.round(t.toFloat / 2.0).toInt + pickQuadrant(RMATa, RMATb, RMATc, RMATd) match { + case 0 => chooseCell(x, y, newT) + case 1 => chooseCell(x + newT, y, newT) + case 2 => chooseCell(x, y + newT, newT) + case 3 => chooseCell(x + newT, y + newT, newT) + } + } + } + + private def pickQuadrant(a: Double, b: Double, c: Double, d: Double): Int = { + if (a + b + c + d != 1.0) { + throw new IllegalArgumentException( + "R-MAT probability parameters sum to " + (a + b + c + d) + + ", should sum to 1.0") + } + val rand = new Random() + val result = rand.nextDouble() + result match { + case x if x < a => 0 // 0 corresponds to quadrant a + case x if (x >= a && x < a + b) => 1 // 1 corresponds to b + case x if (x >= a + b && x < a + b + c) => 2 // 2 corresponds to c + case _ => 3 // 3 corresponds to d + } + } + + /** + * Create `rows` by `cols` grid graph with each vertex connected to its row+1 and col+1 + * neighbors. Vertex ids are assigned in row major order. + * + * @param sc + * the spark context in which to construct the graph + * @param rows + * the number of rows + * @param cols + * the number of columns + * + * @return + * A graph containing vertices with the row and column ids as their attributes and edge values + * as 1.0. + */ + def gridGraph(sc: SparkContext, rows: Int, cols: Int): Graph[(Int, Int), Double] = { + // Convert row column address into vertex ids (row major order) + def sub2ind(r: Int, c: Int): VertexId = (r * cols + c).toLong + + val vertices: RDD[(VertexId, (Int, Int))] = sc.parallelize(0 until rows).flatMap { r => + (0 until cols).map(c => (sub2ind(r, c), (r, c))) + } + val edges: RDD[Edge[Double]] = + vertices + .flatMap { case (_, (r, c)) => + (if (r + 1 < rows) { Seq((sub2ind(r, c), sub2ind(r + 1, c))) } + else { Seq.empty }) ++ + (if (c + 1 < cols) { Seq((sub2ind(r, c), sub2ind(r, c + 1))) } + else { Seq.empty }) + } + .map { case (src, dst) => Edge(src, dst, 1.0) } + Graph[(Int, Int), Double](vertices, edges) + } // end of gridGraph + + /** + * Create a star graph with vertex 0 being the center. + * + * @param sc + * the spark context in which to construct the graph + * @param nverts + * the number of vertices in the star + * + * @return + * A star graph containing `nverts` vertices with vertex 0 being the center vertex. + */ + def starGraph(sc: SparkContext, nverts: Int): Graph[Int, Int] = { + val edges: RDD[(VertexId, VertexId)] = + sc.parallelize(1 until nverts).map(vid => (vid.toLong, 0L)) + Graph.fromEdgeTuples(edges, 1) + } // end of starGraph + +} // end of Graph Generators diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/util/PeriodicGraphCheckpointer.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/util/PeriodicGraphCheckpointer.scala new file mode 100644 index 000000000..e8e2ec1cb --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/util/PeriodicGraphCheckpointer.scala @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.util + +import org.apache.spark.SparkContext +import org.apache.spark.graphframes.graphx.Graph +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.PeriodicCheckpointer + +/** + * This class helps with persisting and checkpointing Graphs. Specifically, it automatically + * handles persisting and (optionally) checkpointing, as well as unpersisting and removing + * checkpoint files. + * + * Users should call update() when a new graph has been created, before the graph has been + * materialized. After updating [[PeriodicGraphCheckpointer]], users are responsible for + * materializing the graph to ensure that persisting and checkpointing actually occur. + * + * When update() is called, this does the following: + * - Persist new graph (if not yet persisted), and put in queue of persisted graphs. + * - Unpersist graphs from queue until there are at most 3 persisted graphs. + * - If using checkpointing and the checkpoint interval has been reached, + * - Checkpoint the new graph, and put in a queue of checkpointed graphs. + * - Remove older checkpoints. + * + * WARNINGS: + * - This class should NOT be copied (since copies may conflict on which Graphs should be + * checkpointed). + * - This class removes checkpoint files once later graphs have been checkpointed. However, + * references to the older graphs will still return isCheckpointed = true. + * + * Example usage: + * {{{ + * val (graph1, graph2, graph3, ...) = ... + * val cp = new PeriodicGraphCheckpointer(2, sc) + * cp.updateGraph(graph1) + * graph1.vertices.count(); graph1.edges.count() + * // persisted: graph1 + * cp.updateGraph(graph2) + * graph2.vertices.count(); graph2.edges.count() + * // persisted: graph1, graph2 + * // checkpointed: graph2 + * cp.updateGraph(graph3) + * graph3.vertices.count(); graph3.edges.count() + * // persisted: graph1, graph2, graph3 + * // checkpointed: graph2 + * cp.updateGraph(graph4) + * graph4.vertices.count(); graph4.edges.count() + * // persisted: graph2, graph3, graph4 + * // checkpointed: graph4 + * cp.updateGraph(graph5) + * graph5.vertices.count(); graph5.edges.count() + * // persisted: graph3, graph4, graph5 + * // checkpointed: graph4 + * }}} + * + * @param checkpointInterval + * Graphs will be checkpointed at this interval. If this interval was set as -1, then + * checkpointing will be disabled. + * @tparam VD + * Vertex descriptor type + * @tparam ED + * Edge descriptor type + */ +private[spark] class PeriodicGraphCheckpointer[VD, ED](checkpointInterval: Int, sc: SparkContext) + extends PeriodicCheckpointer[Graph[VD, ED]](checkpointInterval, sc) { + + override protected def checkpoint(data: Graph[VD, ED]): Unit = data.checkpoint() + + override protected def isCheckpointed(data: Graph[VD, ED]): Boolean = data.isCheckpointed + + override protected def persist(data: Graph[VD, ED]): Unit = { + if (data.vertices.getStorageLevel == StorageLevel.NONE) { + /* We need to use cache because persist does not honor the default storage level requested + * when constructing the graph. Only cache does that. + */ + data.vertices.cache() + } + if (data.edges.getStorageLevel == StorageLevel.NONE) { + data.edges.cache() + } + } + + override protected def unpersist(data: Graph[VD, ED]): Unit = { + val _ = data.unpersist() + () + } + + override protected def getCheckpointFiles(data: Graph[VD, ED]): Iterable[String] = { + data.getCheckpointFiles + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala new file mode 100644 index 000000000..588e667fa --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.util.collection +import org.apache.spark.util.collection.OpenHashSet + +import scala.reflect._ + +/** + * A fast hash map implementation for primitive, non-null keys. This hash map supports insertions + * and updates, but not deletions. This map is about an order of magnitude faster than + * java.util.HashMap, while using much less space overhead. + * + * Under the hood, it uses our OpenHashSet implementation. + */ +private[graphx] class GraphXPrimitiveKeyOpenHashMap[ + @specialized(Long, Int) K: ClassTag, + @specialized(Long, Int, Double) V: ClassTag]( + val keySet: OpenHashSet[K], + var _values: Array[V]) + extends Iterable[(K, V)] + with Serializable { + + /** + * Allocate an OpenHashMap with a fixed initial capacity + */ + def this(initialCapacity: Int) = + this(new OpenHashSet[K](initialCapacity), new Array[V](initialCapacity)) + + /** + * Allocate an OpenHashMap with a default initial capacity, providing a true no-argument + * constructor. + */ + def this() = this(64) + + /** + * Allocate an OpenHashMap with a fixed initial capacity + */ + def this(keySet: OpenHashSet[K]) = this(keySet, new Array[V](keySet.capacity)) + + require(classTag[K] == classTag[Long] || classTag[K] == classTag[Int]) + + private var _oldValues: Array[V] = null + + override def size: Int = keySet.size + + /** Get the value for a given key */ + def apply(k: K): V = { + val pos = keySet.getPos(k) + _values(pos) + } + + /** Get the value for a given key, or returns elseValue if it doesn't exist. */ + def getOrElse(k: K, elseValue: V): V = { + val pos = keySet.getPos(k) + if (pos >= 0) _values(pos) else elseValue + } + + /** Set the value for a key */ + def update(k: K, v: V): Unit = { + val pos = keySet.addWithoutResize(k) & OpenHashSet.POSITION_MASK + _values(pos) = v + keySet.rehashIfNeeded(k, grow, move) + _oldValues = null + } + + /** Set the value for a key */ + def setMerge(k: K, v: V, mergeF: (V, V) => V): Unit = { + val pos = keySet.addWithoutResize(k) + val ind = pos & OpenHashSet.POSITION_MASK + if ((pos & OpenHashSet.NONEXISTENCE_MASK) != 0) { // if first add + _values(ind) = v + } else { + _values(ind) = mergeF(_values(ind), v) + } + keySet.rehashIfNeeded(k, grow, move) + _oldValues = null + } + + /** + * If the key doesn't exist yet in the hash map, set its value to defaultValue; otherwise, set + * its value to mergeValue(oldValue). + * + * @return + * the newly updated value. + */ + def changeValue(k: K, defaultValue: => V, mergeValue: (V) => V): V = { + val pos = keySet.addWithoutResize(k) + if ((pos & OpenHashSet.NONEXISTENCE_MASK) != 0) { + val newValue = defaultValue + _values(pos & OpenHashSet.POSITION_MASK) = newValue + keySet.rehashIfNeeded(k, grow, move) + newValue + } else { + _values(pos) = mergeValue(_values(pos)) + _values(pos) + } + } + + override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] { + var pos = 0 + var nextPair: (K, V) = computeNextPair() + + /** Get the next value we should return from next(), or null if we're finished iterating */ + def computeNextPair(): (K, V) = { + pos = keySet.nextPos(pos) + if (pos >= 0) { + val ret = (keySet.getValue(pos), _values(pos)) + pos += 1 + ret + } else { + null + } + } + + def hasNext: Boolean = nextPair != null + + def next(): (K, V) = { + val pair = nextPair + nextPair = computeNextPair() + pair + } + } + + private def grow(newCapacity: Int): Unit = { + _oldValues = _values + _values = new Array[V](newCapacity) + } + + private def move(oldPos: Int, newPos: Int): Unit = { + _values(newPos) = _oldValues(oldPos) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/util/package-info.java b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/util/package-info.java new file mode 100644 index 000000000..b9f7a5a5c --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/util/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Collections of utilities used by graphx. + */ +package org.apache.spark.graphframes.graphx.util; diff --git a/graphx/src/main/scala/org/apache/spark/graphframes/graphx/util/package.scala b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/util/package.scala new file mode 100644 index 000000000..47e7f1dfc --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphframes/graphx/util/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +/** + * Collections of utilities used by graphx. + */ +package object util diff --git a/graphx/src/test/resources/log4j2.properties b/graphx/src/test/resources/log4j2.properties new file mode 100644 index 000000000..5f8985946 --- /dev/null +++ b/graphx/src/test/resources/log4j2.properties @@ -0,0 +1,31 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file target/unit-tests.log +rootLogger.level = info +rootLogger.appenderRef.file.ref = File + +appender.file.type = File +appender.file.name = File +appender.file.fileName = target/unit-tests.log +appender.file.append = true +appender.file.layout.type = PatternLayout +appender.file.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n%ex + +# Ignore messages below warning level from Jetty, because it's a bit verbose +logger.jetty.name = org.sparkproject.jetty +logger.jetty.level = warn diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/EdgeRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/EdgeRDDSuite.scala new file mode 100644 index 000000000..93101dd2f --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/EdgeRDDSuite.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils +import org.scalatest.funsuite.AnyFunSuite + +class EdgeRDDSuite extends AnyFunSuite with LocalSparkContext { + + test("cache, getStorageLevel") { + // test to see if getStorageLevel returns correct value after caching + withSpark { sc => + val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) + assert(edges.getStorageLevel == StorageLevel.NONE) + edges.cache() + assert(edges.getStorageLevel == StorageLevel.MEMORY_ONLY) + } + } + + test("checkpointing") { + withSpark { sc => + val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) + sc.setCheckpointDir(Utils.createTempDir().getCanonicalPath) + edges.checkpoint() + + // EdgeRDD not yet checkpointed + assert(!edges.isCheckpointed) + assert(!edges.isCheckpointedAndMaterialized) + assert(!edges.partitionsRDD.isCheckpointed) + assert(!edges.partitionsRDD.isCheckpointedAndMaterialized) + + val data = edges.collect().toSeq // force checkpointing + + // EdgeRDD shows up as checkpointed, but internally it is not. + // Only internal partitionsRDD is checkpointed. + assert(edges.isCheckpointed) + assert(!edges.isCheckpointedAndMaterialized) + assert(edges.partitionsRDD.isCheckpointed) + assert(edges.partitionsRDD.isCheckpointedAndMaterialized) + + assert(edges.collect().toSeq === data) // test checkpointed RDD + } + } + + test("count") { + withSpark { sc => + val empty = EdgeRDD.fromEdges(sc.emptyRDD[Edge[Int]]) + assert(empty.count() === 0) + + val edges = List(Edge(0, 1, ()), Edge(1, 2, ()), Edge(2, 0, ())) + val nonempty = EdgeRDD.fromEdges(sc.parallelize(edges)) + assert(nonempty.count() === edges.size) + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/EdgeSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/EdgeSuite.scala new file mode 100644 index 000000000..1c4a8a2a2 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/EdgeSuite.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.scalatest.funsuite.AnyFunSuite + +class EdgeSuite extends AnyFunSuite { + test("compare") { + // descending order + val testEdges: Array[Edge[Int]] = Array( + Edge(0x7fedcba987654321L, -0x7fedcba987654321L, 1), + Edge(0x2345L, 0x1234L, 1), + Edge(0x1234L, 0x5678L, 1), + Edge(0x1234L, 0x2345L, 1), + Edge(-0x7fedcba987654321L, 0x7fedcba987654321L, 1)) + // to ascending order + val sortedEdges = testEdges.sorted(Edge.lexicographicOrdering[Int]) + + for (i <- testEdges.indices) { + assert(sortedEdges(i) == testEdges(testEdges.length - i - 1)) + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/GraphLoaderSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/GraphLoaderSuite.scala new file mode 100644 index 000000000..d3a22baf5 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/GraphLoaderSuite.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.apache.spark.util.Utils +import org.scalatest.funsuite.AnyFunSuite + +import java.io.File +import java.io.FileOutputStream +import java.io.OutputStreamWriter +import java.nio.charset.StandardCharsets + +class GraphLoaderSuite extends AnyFunSuite with LocalSparkContext { + + test("GraphLoader.edgeListFile") { + withSpark { sc => + val tmpDir = Utils.createTempDir() + val graphFile = new File(tmpDir.getAbsolutePath, "graph.txt") + val writer = new OutputStreamWriter(new FileOutputStream(graphFile), StandardCharsets.UTF_8) + for (i <- (1 until 101)) writer.write(s"$i 0\n") + writer.close() + try { + val graph = GraphLoader.edgeListFile(sc, tmpDir.getAbsolutePath) + val neighborAttrSums = + graph.aggregateMessages[Int](ctx => ctx.sendToDst(ctx.srcAttr), _ + _) + assert(neighborAttrSums.collect().toSet === Set((0: VertexId, 100))) + } finally { + Utils.deleteRecursively(tmpDir) + } + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/GraphOpsSuite.scala new file mode 100644 index 000000000..70c08e47c --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/GraphOpsSuite.scala @@ -0,0 +1,228 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.apache.spark.SparkContext +import org.apache.spark.graphframes.graphx.Graph._ +import org.scalatest.funsuite.AnyFunSuite + +class GraphOpsSuite extends AnyFunSuite with LocalSparkContext { + test("joinVertices") { + withSpark { sc => + val vertices = + sc.parallelize(Seq[(VertexId, String)]((1, "one"), (2, "two"), (3, "three")), 2) + val edges = sc.parallelize((Seq(Edge(1, 2, "onetwo")))) + val g: Graph[String, String] = Graph[String, String](vertices, edges) + + val tbl = sc.parallelize(Seq[(VertexId, Int)]((1, 10), (2, 20))) + val g1 = g.joinVertices(tbl) { (_: VertexId, attr: String, u: Int) => attr + u } + + val v = g1.vertices.collect().toSet + assert(v === Set((1, "one10"), (2, "two20"), (3, "three"))) + } + } + + test("collectNeighborIds") { + withSpark { sc => + val graph = getCycleGraph(sc, 100) + val nbrs = graph.collectNeighborIds(EdgeDirection.Either).cache() + assert(nbrs.count() === 100) + assert(graph.numVertices === nbrs.count()) + nbrs.collect().foreach { case (_, nbrs) => assert(nbrs.length === 2) } + nbrs.collect().foreach { case (vid, nbrs) => + val s = nbrs.toSet + assert(s.contains((vid + 1) % 100)) + assert(s.contains(if (vid > 0) vid - 1 else 99)) + } + } + } + + test("removeSelfEdges") { + withSpark { sc => + val edgeArray = Array((1 -> 2), (2 -> 3), (3 -> 3), (4 -> 3), (1 -> 1)) + .map { case (a, b) => + (a.toLong, b.toLong) + } + val correctEdges = edgeArray.filter { case (a, b) => a != b }.toSet + val graph = Graph.fromEdgeTuples(sc.parallelize(edgeArray.toIndexedSeq), 1) + val canonicalizedEdges = graph + .removeSelfEdges() + .edges + .map(e => (e.srcId, e.dstId)) + .collect() + assert(canonicalizedEdges.toSet.size === canonicalizedEdges.length) + assert(canonicalizedEdges.toSet === correctEdges) + } + } + + test("filter") { + withSpark { sc => + val n = 5 + val vertices = sc.parallelize((0 to n).map(x => (x.toLong, x))) + val edges = sc.parallelize((1 to n).map(x => Edge(0L, x.toLong, x))) + val graph: Graph[Int, Int] = Graph[Int, Int](vertices, edges).cache() + val filteredGraph = graph + .filter( + graph => { + val degrees: VertexRDD[Int] = graph.outDegrees + graph.outerJoinVertices(degrees) { (_, _, deg) => deg.getOrElse(0) } + }, + vpred = (_: VertexId, deg: Int) => deg > 0) + .cache() + + val v = filteredGraph.vertices.collect().toSet + assert(v === Set((0, 0))) + + // the map is necessary because of object-reuse in the edge iterator + val e = filteredGraph.edges.map(e => Edge(e.srcId, e.dstId, e.attr)).collect().toSet + assert(e.isEmpty) + } + } + + test("convertToCanonicalEdges") { + withSpark { sc => + val vertices = + sc.parallelize(Seq[(VertexId, String)]((1, "one"), (2, "two"), (3, "three")), 2) + val edges = + sc.parallelize(Seq(Edge(1, 2, 1), Edge(2, 1, 1), Edge(3, 2, 2))) + val g: Graph[String, Int] = Graph[String, Int](vertices, edges) + + val g1 = g.convertToCanonicalEdges() + + val e = g1.edges.collect().toSet + assert(e === Set(Edge(1, 2, 1), Edge(2, 3, 2))) + } + } + + test("collectEdgesCycleDirectionOut") { + withSpark { sc => + val graph = getCycleGraph(sc, 100) + val edges = graph.collectEdges(EdgeDirection.Out).cache() + assert(edges.count() == 100) + edges.collect().foreach { case (_, edges) => assert(edges.length == 1) } + edges.collect().foreach { case (vid, edges) => + val s = edges.toSet + val edgeDstIds = s.map(e => e.dstId) + assert(edgeDstIds.contains((vid + 1) % 100)) + } + } + } + + test("collectEdgesCycleDirectionIn") { + withSpark { sc => + val graph = getCycleGraph(sc, 100) + val edges = graph.collectEdges(EdgeDirection.In).cache() + assert(edges.count() == 100) + edges.collect().foreach { case (_, edges) => assert(edges.length == 1) } + edges.collect().foreach { case (vid, edges) => + val s = edges.toSet + val edgeSrcIds = s.map(e => e.srcId) + assert(edgeSrcIds.contains(if (vid > 0) vid - 1 else 99)) + } + } + } + + test("collectEdgesCycleDirectionEither") { + withSpark { sc => + val graph = getCycleGraph(sc, 100) + val edges = graph.collectEdges(EdgeDirection.Either).cache() + assert(edges.count() == 100) + edges.collect().foreach { case (_, edges) => assert(edges.length == 2) } + edges.collect().foreach { case (vid, edges) => + val s = edges.toSet + val edgeIds = s.map(e => if (vid != e.srcId) e.srcId else e.dstId) + assert(edgeIds.contains((vid + 1) % 100)) + assert(edgeIds.contains(if (vid > 0) vid - 1 else 99)) + } + } + } + + test("collectEdgesChainDirectionOut") { + withSpark { sc => + val graph = getChainGraph(sc, 50) + val edges = graph.collectEdges(EdgeDirection.Out).cache() + assert(edges.count() == 49) + edges.collect().foreach { case (_, edges) => assert(edges.length == 1) } + edges.collect().foreach { case (vid, edges) => + val s = edges.toSet + val edgeDstIds = s.map(e => e.dstId) + assert(edgeDstIds.contains(vid + 1)) + } + } + } + + test("collectEdgesChainDirectionIn") { + withSpark { sc => + val graph = getChainGraph(sc, 50) + val edges = graph.collectEdges(EdgeDirection.In).cache() + // We expect only 49 because collectEdges does not return vertices that do + // not have any edges in the specified direction. + assert(edges.count() == 49) + edges.collect().foreach { case (_, edges) => assert(edges.length == 1) } + edges.collect().foreach { case (vid, edges) => + val s = edges.toSet + val edgeDstIds = s.map(e => e.srcId) + assert(edgeDstIds.contains((vid - 1) % 100)) + } + } + } + + test("collectEdgesChainDirectionEither") { + withSpark { sc => + val graph = getChainGraph(sc, 50) + val edges = graph.collectEdges(EdgeDirection.Either).cache() + // We expect only 49 because collectEdges does not return vertices that do + // not have any edges in the specified direction. + assert(edges.count() === 50) + edges.collect().foreach { case (vid, edges) => + if (vid > 0 && vid < 49) { + assert(edges.length == 2) + } else { + assert(edges.length == 1) + } + } + edges.collect().foreach { case (vid, edges) => + val s = edges.toSet + val edgeIds = s.map(e => if (vid != e.srcId) e.srcId else e.dstId) + if (vid == 0) { assert(edgeIds.contains(1)) } + else if (vid == 49) { assert(edgeIds.contains(48)) } + else { + assert(edgeIds.contains(vid + 1)) + assert(edgeIds.contains(vid - 1)) + } + } + } + } + + private def getCycleGraph(sc: SparkContext, numVertices: Int): Graph[Double, Int] = { + val cycle = (0 until numVertices).map(x => (x, (x + 1) % numVertices)) + getGraphFromSeq(sc, cycle) + } + + private def getChainGraph(sc: SparkContext, numVertices: Int): Graph[Double, Int] = { + val chain = (0 until numVertices - 1).map(x => (x, (x + 1))) + getGraphFromSeq(sc, chain) + } + + private def getGraphFromSeq( + sc: SparkContext, + seq: IndexedSeq[(Int, Int)]): Graph[Double, Int] = { + val rawEdges = sc.parallelize(seq, 3).map { case (s, d) => (s.toLong, d.toLong) } + Graph.fromEdgeTuples(rawEdges, 1.0).cache() + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/GraphSuite.scala new file mode 100644 index 000000000..eb550a1f7 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/GraphSuite.scala @@ -0,0 +1,448 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.apache.spark.SparkContext +import org.apache.spark.graphframes.graphx.Graph._ +import org.apache.spark.graphframes.graphx.PartitionStrategy._ +import org.apache.spark.rdd._ +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils +import org.scalatest.funsuite.AnyFunSuite + +class GraphSuite extends AnyFunSuite with LocalSparkContext { + + def starGraph(sc: SparkContext, n: Int): Graph[String, Int] = { + Graph.fromEdgeTuples[String]( + sc.parallelize((1L to n.toLong).map(x => (0L: VertexId, x: VertexId)), 3), + "v") + } + + test("Graph.fromEdgeTuples") { + withSpark { sc => + val ring = (0L to 100L).zip((1L to 99L) :+ 0L) + val doubleRing = ring ++ ring + val graph = Graph.fromEdgeTuples(sc.parallelize(doubleRing), 1) + assert(graph.edges.count() === doubleRing.size) + assert(graph.edges.collect().forall(e => e.attr == 1)) + + // uniqueEdges option should uniquify edges and store duplicate count in edge attributes + val uniqueGraph = Graph.fromEdgeTuples(sc.parallelize(doubleRing), 1, Some(RandomVertexCut)) + assert(uniqueGraph.edges.count() === ring.size) + assert(uniqueGraph.edges.collect().forall(e => e.attr == 2)) + } + } + + test("Graph.fromEdges") { + withSpark { sc => + val ring = (0L to 100L).zip((1L to 99L) :+ 0L).map { case (a, b) => Edge(a, b, 1) } + val graph = Graph.fromEdges(sc.parallelize(ring), 1.0f) + assert(graph.edges.count() === ring.size) + } + } + + test("Graph.apply") { + withSpark { sc => + val rawEdges = (0L to 98L).zip((1L to 99L) :+ 0L) + val edges: RDD[Edge[Int]] = sc.parallelize(rawEdges).map { case (s, t) => Edge(s, t, 1) } + val vertices: RDD[(VertexId, Boolean)] = + sc.parallelize((0L until 10L).map(id => (id, true))) + val graph = Graph(vertices, edges, false) + assert(graph.edges.count() === rawEdges.size) + // Vertices not explicitly provided but referenced by edges should be created automatically + assert(graph.vertices.count() === 100) + graph.triplets.collect().foreach { et => + assert((et.srcId < 10 && et.srcAttr) || (et.srcId >= 10 && !et.srcAttr)) + assert((et.dstId < 10 && et.dstAttr) || (et.dstId >= 10 && !et.dstAttr)) + } + } + } + + test("triplets") { + withSpark { sc => + val n = 5 + val star = starGraph(sc, n) + assert( + star.triplets.map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)).collect().toSet + === (1L to n.toLong).map(x => (0L: VertexId, x: VertexId, "v", "v")).toSet) + } + } + + test("partitionBy") { + withSpark { sc => + def mkGraph(edges: List[(Long, Long)]): Graph[Int, Int] = { + Graph.fromEdgeTuples(sc.parallelize(edges, 2), 0) + } + def nonemptyParts(graph: Graph[Int, Int]): RDD[List[Edge[Int]]] = { + graph.edges.partitionsRDD + .mapPartitions { iter => + Iterator(iter.next()._2.iterator.toList) + } + .filter(_.nonEmpty) + } + val identicalEdges = List((0L, 1L), (0L, 1L)) + val canonicalEdges = List((0L, 1L), (1L, 0L)) + val sameSrcEdges = List((0L, 1L), (0L, 2L)) + + // The two edges start out in different partitions + for (edges <- List(identicalEdges, canonicalEdges, sameSrcEdges)) { + assert(nonemptyParts(mkGraph(edges)).count() === 2) + } + // partitionBy(RandomVertexCut) puts identical edges in the same partition + assert(nonemptyParts(mkGraph(identicalEdges).partitionBy(RandomVertexCut)).count() === 1) + // partitionBy(EdgePartition1D) puts same-source edges in the same partition + assert(nonemptyParts(mkGraph(sameSrcEdges).partitionBy(EdgePartition1D)).count() === 1) + // partitionBy(CanonicalRandomVertexCut) puts edges that are identical modulo direction into + // the same partition + assert( + nonemptyParts(mkGraph(canonicalEdges).partitionBy(CanonicalRandomVertexCut)) + .count() === 1) + // partitionBy(EdgePartition2D) puts identical edges in the same partition + assert(nonemptyParts(mkGraph(identicalEdges).partitionBy(EdgePartition2D)).count() === 1) + + // partitionBy(EdgePartition2D) ensures that vertices need only be replicated to 2 * sqrt(p) + // partitions + val n = 100 + val p = 100 + val verts = 1L to n.toLong + val graph = Graph.fromEdgeTuples( + sc.parallelize( + verts.flatMap(x => + verts.withFilter(y => y % x == 0).map(y => (x: VertexId, y: VertexId))), + p), + 0) + assert(graph.edges.partitions.length === p) + val partitionedGraph = graph.partitionBy(EdgePartition2D) + assert(graph.edges.partitions.length === p) + val bound = 2.0 * math.sqrt(p.toDouble) + // Each vertex should be replicated to at most 2 * sqrt(p) partitions + val partitionSets = partitionedGraph.edges.partitionsRDD + .mapPartitions { iter => + val part = iter.next()._2 + Iterator((part.iterator.flatMap(e => Iterator(e.srcId, e.dstId))).toSet) + } + .collect() + if (!verts.forall(id => partitionSets.count(_.contains(id)) <= bound)) { + val numFailures = verts.count(id => partitionSets.count(_.contains(id)) > bound) + val failure = verts.maxBy(id => partitionSets.count(_.contains(id))) + fail( + ("Replication bound test failed for %d/%d vertices. " + + "Example: vertex %d replicated to %d (> %f) partitions.") + .format(numFailures, n, failure, partitionSets.count(_.contains(failure)), bound)) + } + // This should not be true for the default hash partitioning + val partitionSetsUnpartitioned = graph.edges.partitionsRDD + .mapPartitions { iter => + val part = iter.next()._2 + Iterator((part.iterator.flatMap(e => Iterator(e.srcId, e.dstId))).toSet) + } + .collect() + assert(verts.exists(id => partitionSetsUnpartitioned.count(_.contains(id)) > bound)) + + // Forming triplets view + val g = Graph[String, Int]( + sc.parallelize(List((0L, "a"), (1L, "b"), (2L, "c"))), + sc.parallelize(List(Edge(0L, 1L, 1), Edge(0L, 2L, 1)), 2)) + assert( + g.triplets.collect().map(_.toTuple).toSet === + Set(((0L, "a"), (1L, "b"), 1), ((0L, "a"), (2L, "c"), 1))) + val gPart = g.partitionBy(EdgePartition2D) + assert( + gPart.triplets.collect().map(_.toTuple).toSet === + Set(((0L, "a"), (1L, "b"), 1), ((0L, "a"), (2L, "c"), 1))) + } + } + + test("mapVertices") { + withSpark { sc => + val n = 5 + val star = starGraph(sc, n) + // mapVertices preserving type + val mappedVAttrs = star.mapVertices((_, attr) => attr + "2") + assert( + mappedVAttrs.vertices.collect().toSet === (0L to n.toLong) + .map(x => (x: VertexId, "v2")) + .toSet) + // mapVertices changing type + val mappedVAttrs2 = star.mapVertices((_, attr) => attr.length) + assert( + mappedVAttrs2.vertices.collect().toSet === (0L to n.toLong) + .map(x => (x: VertexId, 1)) + .toSet) + } + } + + test("mapVertices changing type with same erased type") { + withSpark { sc => + val vertices = sc.parallelize( + Seq[(Long, Option[java.lang.Integer])]((1L, Some(1)), (2L, Some(2)), (3L, Some(3)))) + val edges = sc.parallelize(Seq(Edge(1L, 2L, 0), Edge(2L, 3L, 0), Edge(3L, 1L, 0))) + val graph0 = Graph[Option[java.lang.Integer], Int](vertices, edges) + // Trigger initial vertex replication + graph0.triplets.foreach(_ => {}) + // Change type of replicated vertices, but preserve erased type + val graph1 = graph0.mapVertices { case (_, integerOpt) => + integerOpt.map((x: java.lang.Integer) => x.toDouble: java.lang.Double) + } + // Access replicated vertices, exposing the erased type + val graph2 = graph1.mapTriplets(t => t.srcAttr.get) + assert(graph2.edges.map(_.attr).collect().toSet === Set[java.lang.Double](1.0, 2.0, 3.0)) + } + } + + test("mapEdges") { + withSpark { sc => + val n = 3 + val star = starGraph(sc, n) + val starWithEdgeAttrs = star.mapEdges(e => e.dstId) + + val edges = starWithEdgeAttrs.edges.collect() + assert(edges.length === n) + assert(edges.toSet === (1 to n).map(x => Edge(0L, x.toLong, x)).toSet) + } + } + + test("mapTriplets") { + withSpark { sc => + val n = 5 + val star = starGraph(sc, n) + assert( + star.mapTriplets(et => et.srcAttr + et.dstAttr).edges.collect().toSet === + (1L to n).map(x => Edge(0L, x.toLong, "vv")).toSet) + } + } + + test("reverse") { + withSpark { sc => + val n = 5 + val star = starGraph(sc, n) + assert(star.reverse.outDegrees.collect().toSet === (1 to n).map(x => (x.toLong, 1)).toSet) + } + } + + test("reverse with join elimination") { + withSpark { sc => + val vertices: RDD[(VertexId, Int)] = sc.parallelize(Seq((1L, 1), (2L, 2))) + val edges: RDD[Edge[Int]] = sc.parallelize(Seq(Edge(1L, 2L, 0))) + val graph = Graph[Int, Int](vertices, edges).reverse + val result = GraphXUtils.mapReduceTriplets[Int, Int, Int]( + graph, + et => Iterator((et.dstId, et.srcAttr)), + _ + _) + assert(result.collect().toSet === Set((1L, 2))) + } + } + + test("subgraph") { + withSpark { sc => + // Create a star graph of 10 vertices. + val n = 10 + val star = starGraph(sc, n) + // Take only vertices whose vids are even + val subgraph = star.subgraph(vpred = (vid, _) => vid % 2 == 0) + + // We should have 5 vertices. + assert(subgraph.vertices.collect().toSet === (0 to n by 2).map(x => (x, "v")).toSet) + + // And 4 edges. + assert( + subgraph.edges.map(_.copy()).collect().toSet === + (2 to n by 2).map(x => Edge(0, x.toLong, 1)).toSet) + } + } + + test("mask") { + withSpark { sc => + val n = 5 + val vertices = sc.parallelize((0 to n).map(x => (x.toLong, x))) + val edges = sc.parallelize((1 to n).map(x => Edge(0L, x.toLong, x))) + val graph: Graph[Int, Int] = Graph[Int, Int](vertices, edges).cache() + + val subgraph = graph + .subgraph(e => e.dstId != 4L, (vid, _) => vid != 3L) + .mapVertices((_, _) => -1) + .mapEdges(_ => -1) + + val projectedGraph = graph.mask(subgraph) + + val v = projectedGraph.vertices.collect().toSet + assert(v === Set((0, 0), (1, 1), (2, 2), (4, 4), (5, 5))) + + // the map is necessary because of object-reuse in the edge iterator + val e = projectedGraph.edges.map(e => Edge(e.srcId, e.dstId, e.attr)).collect().toSet + assert(e === Set(Edge(0, 1, 1), Edge(0, 2, 2), Edge(0, 5, 5))) + + } + } + + test("groupEdges") { + withSpark { sc => + val n = 5 + val star = starGraph(sc, n) + val doubleStar = Graph.fromEdgeTuples( + sc.parallelize((1 to n).flatMap(x => List((0L, x.toLong), (0L, x.toLong))), 1), + "v") + val star2 = doubleStar.groupEdges { (a, _) => a } + assert( + star2.edges.collect().toArray.sorted(Edge.lexicographicOrdering[Int]) === + star.edges.collect().toArray.sorted(Edge.lexicographicOrdering[Int])) + assert(star2.vertices.collect().toSet === star.vertices.collect().toSet) + } + } + + test("aggregateMessages") { + withSpark { sc => + val n = 5 + val agg = starGraph(sc, n).aggregateMessages[String]( + ctx => { + if (ctx.dstAttr != null) { + throw new Exception( + "expected ctx.dstAttr to be null due to TripletFields, but it was " + ctx.dstAttr) + } + ctx.sendToDst(ctx.srcAttr) + }, + _ + _, + TripletFields.Src) + assert(agg.collect().toSet === (1L to n.toLong).map(x => (x: VertexId, "v")).toSet) + } + } + + test("outerJoinVertices") { + withSpark { sc => + val n = 5 + val reverseStar = starGraph(sc, n).reverse.cache() + // outerJoinVertices changing type + val reverseStarDegrees = reverseStar.outerJoinVertices(reverseStar.outDegrees) { + (_, _, bOpt) => bOpt.getOrElse(0) + } + val neighborDegreeSums = GraphXUtils + .mapReduceTriplets[Int, Int, Int]( + reverseStarDegrees, + et => Iterator((et.srcId, et.dstAttr), (et.dstId, et.srcAttr)), + (a: Int, b: Int) => a + b) + .collect() + .toSet + assert(neighborDegreeSums === Set((0L, n)) ++ (1 to n).map(x => (x.toLong, 0))) + // outerJoinVertices preserving type + val messages = reverseStar.vertices.mapValues { (vid, _) => vid.toString } + val newReverseStar = + reverseStar.outerJoinVertices(messages) { (_, a, bOpt) => a + bOpt.getOrElse("") } + assert( + newReverseStar.vertices.map(_._2).collect().toSet === + (0 to n).map(x => "v%d".format(x)).toSet) + } + } + + test("more edge partitions than vertex partitions") { + withSpark { sc => + val verts = sc.parallelize(List((1: VertexId, "a"), (2: VertexId, "b")), 1) + val edges = sc.parallelize(List(Edge(1, 2, 0), Edge(2, 1, 0)), 2) + val graph = Graph[String, Int](verts, edges) + val triplets = graph.triplets + .map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)) + .collect() + .toSet + assert( + triplets === + Set((1: VertexId, 2: VertexId, "a", "b"), (2: VertexId, 1: VertexId, "b", "a"))) + } + } + + test("checkpoint") { + val checkpointDir = Utils.createTempDir() + withSpark { sc => + sc.setCheckpointDir(checkpointDir.getAbsolutePath) + val ring = (0L to 100L).zip((1L to 99L) :+ 0L).map { case (a, b) => Edge(a, b, 1) } + val rdd = sc.parallelize(ring) + val graph = Graph.fromEdges(rdd, 1.0f) + assert(!graph.isCheckpointed) + assert(graph.getCheckpointFiles.size === 0) + graph.checkpoint() + graph.edges.map(_.attr).count() + graph.vertices.map(_._2).count() + + val edgesDependencies = graph.edges.partitionsRDD.dependencies + val verticesDependencies = graph.vertices.partitionsRDD.dependencies + assert(edgesDependencies.forall(_.rdd.isInstanceOf[CheckpointRDD[_]])) + assert(verticesDependencies.forall(_.rdd.isInstanceOf[CheckpointRDD[_]])) + assert(graph.isCheckpointed) + assert(graph.getCheckpointFiles.size === 2) + } + } + + test("cache, getStorageLevel") { + // test to see if getStorageLevel returns correct value + withSpark { sc => + val verts = sc.parallelize(List((1: VertexId, "a"), (2: VertexId, "b")), 1) + val edges = sc.parallelize(List(Edge(1, 2, 0), Edge(2, 1, 0)), 2) + val graph = Graph(verts, edges, "", StorageLevel.MEMORY_ONLY, StorageLevel.MEMORY_ONLY) + // Note: Before caching, graph.vertices is cached, but graph.edges is not (but graph.edges' + // parent RDD is cached). + graph.cache() + assert(graph.vertices.getStorageLevel == StorageLevel.MEMORY_ONLY) + assert(graph.edges.getStorageLevel == StorageLevel.MEMORY_ONLY) + } + } + + test("non-default number of edge partitions") { + val n = 10 + val defaultParallelism = 3 + val numEdgePartitions = 4 + assert(defaultParallelism != numEdgePartitions) + val conf = new org.apache.spark.SparkConf() + .set("spark.default.parallelism", defaultParallelism.toString) + val sc = new SparkContext("local", "test", conf) + try { + val edges = sc.parallelize((1 to n).map(x => (x.toLong, 0L)), numEdgePartitions) + val graph = Graph.fromEdgeTuples(edges, 1) + val neighborAttrSums = GraphXUtils.mapReduceTriplets[Int, Int, Int]( + graph, + et => Iterator((et.dstId, et.srcAttr)), + _ + _) + assert(neighborAttrSums.collect().toSet === Set((0L, n))) + } finally { + sc.stop() + } + } + + test("unpersist graph RDD") { + withSpark { sc => + val vert = sc.parallelize(List((1L, "a"), (2L, "b"), (3L, "c")), 1) + val edges = sc.parallelize(List(Edge[Long](1L, 2L), Edge[Long](1L, 3L)), 1) + val g0 = Graph[String, Long](vert, edges) + val g = g0.partitionBy(PartitionStrategy.EdgePartition2D, 2) + val cc = g.connectedComponents() + assert(sc.getPersistentRDDs.nonEmpty) + cc.unpersist(blocking = true) + g.unpersist(blocking = true) + g0.unpersist(blocking = true) + vert.unpersist(blocking = true) + edges.unpersist(blocking = true) + assert(sc.getPersistentRDDs.isEmpty) + } + } + + test("SPARK-14219: pickRandomVertex") { + withSpark { sc => + val vert = sc.parallelize(List((1L, "a")), 1) + val edges = sc.parallelize(List(Edge[Long](1L, 1L)), 1) + val g0 = Graph[String, Long](vert, edges) + assert(g0.pickRandomVertex() === 1L) + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/LocalSparkContext.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/LocalSparkContext.scala new file mode 100644 index 000000000..68b8e9a71 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/LocalSparkContext.scala @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext + +/** + * Provides a method to run tests against a `SparkContext` variable that is correctly stopped + * after each test. + */ +trait LocalSparkContext { + + /** Runs `f` on a new SparkContext and ensures that it is stopped afterwards. */ + def withSpark[T](f: SparkContext => T): T = { + val conf = new SparkConf() + GraphXUtils.registerKryoClasses(conf) + val sc = new SparkContext("local", "test", conf) + try { + f(sc) + } finally { + sc.stop() + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/PregelSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/PregelSuite.scala new file mode 100644 index 000000000..c614e8739 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/PregelSuite.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.scalatest.funsuite.AnyFunSuite + +class PregelSuite extends AnyFunSuite with LocalSparkContext { + + test("1 iteration") { + withSpark { sc => + val n = 5 + val starEdges = (1 to n).map(x => (0: VertexId, x: VertexId)) + val star = Graph.fromEdgeTuples(sc.parallelize(starEdges, 3), "v").cache() + val result = Pregel(star, 0)( + (_, attr, _) => attr, + _ => Iterator.empty, + (_: Int, _: Int) => throw new Exception("mergeMsg run unexpectedly")) + assert(result.vertices.collect().toSet === star.vertices.collect().toSet) + } + } + + test("chain propagation") { + withSpark { sc => + val n = 5 + val chain = Graph + .fromEdgeTuples( + sc.parallelize((1 until n).map(x => (x: VertexId, x + 1: VertexId)), 3), + 0) + .cache() + assert(chain.vertices.collect().toSet === (1 to n).map(x => (x: VertexId, 0)).toSet) + val chainWithSeed = chain.mapVertices { (vid, _) => if (vid == 1) 1 else 0 }.cache() + assert( + chainWithSeed.vertices.collect().toSet === + Set((1: VertexId, 1)) ++ (2 to n).map(x => (x: VertexId, 0)).toSet) + val result = Pregel(chainWithSeed, 0)( + (_, attr, msg) => math.max(msg, attr), + et => if (et.dstAttr != et.srcAttr) Iterator((et.dstId, et.srcAttr)) else Iterator.empty, + (a: Int, b: Int) => math.max(a, b)) + assert( + result.vertices.collect().toSet === + chain.vertices.mapValues { (_, attr) => attr + 1 }.collect().toSet) + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/VertexRDDSuite.scala new file mode 100644 index 000000000..da656f5c4 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/VertexRDDSuite.scala @@ -0,0 +1,250 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx + +import org.apache.spark.HashPartitioner +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils +import org.scalatest.funsuite.AnyFunSuite + +class VertexRDDSuite extends AnyFunSuite with LocalSparkContext { + + private def vertices(sc: SparkContext, n: Int) = { + VertexRDD(sc.parallelize((0 to n).map(x => (x.toLong, x)), 5)) + } + + test("filter") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + val evens = verts.filter(q => ((q._2 % 2) == 0)) + assert(evens.count() === (0 to n).count(_ % 2 == 0)) + } + } + + test("mapValues") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + val negatives = verts.mapValues(x => -x).cache() // Allow joining b with a derived RDD of b + assert(negatives.count() === n + 1) + } + } + + test("minus") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 75, 2).map(i => (i.toLong, 0))).cache() + val vertexB = VertexRDD(sc.parallelize(25 until 100, 2).map(i => (i.toLong, 1))).cache() + val vertexC = vertexA.minus(vertexB) + assert(vertexC.map(_._1).collect().toSet === (0 until 25).toSet) + } + } + + test("minus with RDD[(VertexId, VD)]") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 75, 2).map(i => (i.toLong, 0))).cache() + val vertexB: RDD[(VertexId, Int)] = + sc.parallelize(25 until 100, 2).map(i => (i.toLong, 1)).cache() + val vertexC = vertexA.minus(vertexB) + assert(vertexC.map(_._1).collect().toSet === (0 until 25).toSet) + } + } + + test("minus with non-equal number of partitions") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 75, 5).map(i => (i.toLong, 0))) + val vertexB = VertexRDD(sc.parallelize(50 until 100, 2).map(i => (i.toLong, 1))) + assert(vertexA.partitions.length != vertexB.partitions.length) + val vertexC = vertexA.minus(vertexB) + assert(vertexC.map(_._1).collect().toSet === (0 until 50).toSet) + } + } + + test("diff") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n).cache() + val flipEvens = verts.mapValues(x => if (x % 2 == 0) -x else x).cache() + // diff should keep only the changed vertices + assert(verts.diff(flipEvens).map(_._2).collect().toSet === (2 to n by 2).map(-_).toSet) + // diff should keep the vertex values from `other` + assert(flipEvens.diff(verts).map(_._2).collect().toSet === (2 to n by 2).toSet) + } + } + + test("diff with RDD[(VertexId, VD)]") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n).cache() + val flipEvens: RDD[(VertexId, Int)] = + sc.parallelize(0L to 100L) + .map(id => if (id % 2 == 0) (id, -id.toInt) else (id, id.toInt)) + .cache() + // diff should keep only the changed vertices + assert(verts.diff(flipEvens).map(_._2).collect().toSet === (2 to n by 2).map(-_).toSet) + } + } + + test("diff vertices with non-equal number of partitions") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 24, 3).map(i => (i.toLong, 0))) + val vertexB = VertexRDD(sc.parallelize(8 until 16, 2).map(i => (i.toLong, 1))) + assert(vertexA.partitions.length != vertexB.partitions.length) + val vertexC = vertexA.diff(vertexB) + assert(vertexC.map(_._1).collect().toSet === (8 until 16).toSet) + } + } + + test("leftJoin") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n).cache() + val evens = verts.filter(q => ((q._2 % 2) == 0)).cache() + // leftJoin with another VertexRDD + assert( + verts.leftJoin(evens) { (_, a, bOpt) => a - bOpt.getOrElse(0) }.collect().toSet === + (0 to n by 2).map(x => (x.toLong, 0)).toSet ++ (1 to n by 2) + .map(x => (x.toLong, x)) + .toSet) + // leftJoin with an RDD + val evensRDD = evens.map(identity) + assert( + verts.leftJoin(evensRDD) { (_, a, bOpt) => a - bOpt.getOrElse(0) }.collect().toSet === + (0 to n by 2).map(x => (x.toLong, 0)).toSet ++ (1 to n by 2) + .map(x => (x.toLong, x)) + .toSet) + } + } + + test("leftJoin vertices with non-equal number of partitions") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 100, 2).map(i => (i.toLong, 1))) + val vertexB = + VertexRDD(vertexA.filter(v => v._1 % 2 == 0).partitionBy(new HashPartitioner(3))) + assert(vertexA.partitions.length != vertexB.partitions.length) + val vertexC = vertexA.leftJoin(vertexB) { (_, old, newOpt) => + old - newOpt.getOrElse(0) + } + assert(vertexC.filter(v => v._2 != 0).map(_._1).collect().toSet == (1 to 99 by 2).toSet) + } + } + + test("innerJoin") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n).cache() + val evens = verts.filter(q => ((q._2 % 2) == 0)).cache() + // innerJoin with another VertexRDD + assert( + verts.innerJoin(evens) { (_, a, b) => a - b }.collect().toSet === + (0 to n by 2).map(x => (x.toLong, 0)).toSet) + // innerJoin with an RDD + val evensRDD = evens.map(identity) + assert( + verts.innerJoin(evensRDD) { (_, a, b) => a - b }.collect().toSet === + (0 to n by 2).map(x => (x.toLong, 0)).toSet) + } + } + + test("innerJoin vertices with the non-equal number of partitions") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 100, 2).map(i => (i.toLong, 1))) + val vertexB = + VertexRDD(vertexA.filter(v => v._1 % 2 == 0).partitionBy(new HashPartitioner(3))) + assert(vertexA.partitions.length != vertexB.partitions.length) + val vertexC = vertexA.innerJoin(vertexB) { (_, old, newVal) => + old - newVal + } + assert(vertexC.filter(v => v._2 == 0).map(_._1).collect().toSet == (0 to 98 by 2).toSet) + } + } + + test("aggregateUsingIndex") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + val messageTargets = (0 to n) ++ (0 to n by 2) + val messages = sc.parallelize(messageTargets.map(x => (x.toLong, 1))) + assert( + verts.aggregateUsingIndex[Int](messages, _ + _).collect().toSet === + (0 to n).map(x => (x.toLong, if (x % 2 == 0) 2 else 1)).toSet) + } + } + + test("mergeFunc") { + // test to see if the mergeFunc is working correctly + withSpark { sc => + val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3))) + val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) + val rdd = VertexRDD(verts, edges, 0, (a: Int, b: Int) => a + b) + // test merge function + assert(rdd.collect().toSet == Set((0L, 0), (1L, 3), (2L, 9))) + } + } + + test("cache, getStorageLevel") { + // test to see if getStorageLevel returns correct value after caching + withSpark { sc => + val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3))) + val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) + val rdd = VertexRDD(verts, edges, 0, (a: Int, b: Int) => a + b) + assert(rdd.getStorageLevel == StorageLevel.NONE) + rdd.cache() + assert(rdd.getStorageLevel == StorageLevel.MEMORY_ONLY) + } + } + + test("checkpoint") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + sc.setCheckpointDir(Utils.createTempDir().getCanonicalPath) + verts.checkpoint() + + // VertexRDD not yet checkpointed + assert(!verts.isCheckpointed) + assert(!verts.isCheckpointedAndMaterialized) + assert(!verts.partitionsRDD.isCheckpointed) + assert(!verts.partitionsRDD.isCheckpointedAndMaterialized) + + val data = verts.collect().toSeq // force checkpointing + + // VertexRDD shows up as checkpointed, but internally it is not. + // Only internal partitionsRDD is checkpointed. + assert(verts.isCheckpointed) + assert(!verts.isCheckpointedAndMaterialized) + assert(verts.partitionsRDD.isCheckpointed) + assert(verts.partitionsRDD.isCheckpointedAndMaterialized) + + assert(verts.collect().toSeq === data) // test checkpointed RDD + } + } + + test("count") { + withSpark { sc => + val empty = VertexRDD(sc.emptyRDD[(Long, Unit)]) + assert(empty.count() === 0) + + val n = 100 + val nonempty = vertices(sc, n) + assert(nonempty.count() === n + 1) + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/impl/EdgePartitionSuite.scala new file mode 100644 index 000000000..04da8e25c --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/impl/EdgePartitionSuite.scala @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.impl + +import org.apache.spark.SparkConf +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.serializer.KryoSerializer +import org.scalatest.funsuite.AnyFunSuite + +import scala.reflect.ClassTag + +class EdgePartitionSuite extends AnyFunSuite { + + def makeEdgePartition[A: ClassTag](xs: Iterable[(Int, Int, A)]): EdgePartition[A, Int] = { + val builder = new EdgePartitionBuilder[A, Int] + for ((src, dst, attr) <- xs) { builder.add(src: VertexId, dst: VertexId, attr) } + builder.toEdgePartition + } + + test("reverse") { + val edges = List(Edge(0, 1, 0), Edge(1, 2, 0), Edge(2, 0, 0)) + val reversedEdges = List(Edge(0, 2, 0), Edge(1, 0, 0), Edge(2, 1, 0)) + val builder = new EdgePartitionBuilder[Int, Nothing] + for (e <- edges) { + builder.add(e.srcId, e.dstId, e.attr) + } + val edgePartition = builder.toEdgePartition + assert(edgePartition.reverse.iterator.map(_.copy()).toList === reversedEdges) + assert(edgePartition.reverse.reverse.iterator.map(_.copy()).toList === edges) + } + + test("map") { + val edges = List(Edge(0, 1, 0), Edge(1, 2, 0), Edge(2, 0, 0)) + val builder = new EdgePartitionBuilder[Int, Nothing] + for (e <- edges) { + builder.add(e.srcId, e.dstId, e.attr) + } + val edgePartition = builder.toEdgePartition + assert( + edgePartition.map(e => e.srcId + e.dstId).iterator.map(_.copy()).toList === + edges.map(e => e.copy(attr = e.srcId + e.dstId))) + } + + test("filter") { + val edges = List(Edge(0, 1, 0), Edge(0, 2, 0), Edge(2, 0, 0)) + val builder = new EdgePartitionBuilder[Int, Int] + for (e <- edges) { + builder.add(e.srcId, e.dstId, e.attr) + } + val edgePartition = builder.toEdgePartition + val filtered = edgePartition.filter(et => et.srcId == 0, (vid, _) => vid == 0 || vid == 1) + assert(filtered.tripletIterator().toList.map(et => (et.srcId, et.dstId)) === List((0L, 1L))) + } + + test("groupEdges") { + val edges = List( + Edge(0, 1, 1), + Edge(1, 2, 2), + Edge(2, 0, 4), + Edge(0, 1, 8), + Edge(1, 2, 16), + Edge(2, 0, 32)) + val groupedEdges = List(Edge(0, 1, 9), Edge(1, 2, 18), Edge(2, 0, 36)) + val builder = new EdgePartitionBuilder[Int, Nothing] + for (e <- edges) { + builder.add(e.srcId, e.dstId, e.attr) + } + val edgePartition = builder.toEdgePartition + assert(edgePartition.groupEdges(_ + _).iterator.map(_.copy()).toList === groupedEdges) + } + + test("innerJoin") { + val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) + val bList = List((0, 1, 0), (1, 0, 0), (1, 1, 0), (3, 4, 0), (5, 5, 0)) + val a = makeEdgePartition(aList) + val b = makeEdgePartition(bList) + + assert( + a.innerJoin(b) { (_, _, a, _) => a }.iterator.map(_.copy()).toList === + List(Edge(0, 1, 0), Edge(1, 0, 0), Edge(5, 5, 0))) + } + + test("isActive, numActives, replaceActives") { + val ep = new EdgePartitionBuilder[Nothing, Nothing].toEdgePartition + .withActiveSet(Iterator(0L, 2L, 0L)) + assert(ep.isActive(0)) + assert(!ep.isActive(1)) + assert(ep.isActive(2)) + assert(!ep.isActive(-1)) + assert(ep.numActives == Some(2)) + } + + test("tripletIterator") { + val builder = new EdgePartitionBuilder[Int, Int] + builder.add(1, 2, 0) + builder.add(1, 3, 0) + builder.add(1, 4, 0) + val ep = builder.toEdgePartition + val result = ep.tripletIterator().toList.map(et => (et.srcId, et.dstId)) + assert(result === Seq((1, 2), (1, 3), (1, 4))) + } + + test("serialization") { + val aList = List((0, 1, 1), (1, 0, 2), (1, 2, 3), (5, 4, 4), (5, 5, 5)) + val a: EdgePartition[Int, Int] = makeEdgePartition(aList) + val javaSer = new JavaSerializer(new SparkConf()) + val conf = new SparkConf() + GraphXUtils.registerKryoClasses(conf) + val kryoSer = new KryoSerializer(conf) + + for (ser <- List(javaSer, kryoSer); s = ser.newInstance()) { + val aSer: EdgePartition[Int, Int] = s.deserialize[EdgePartition[Int, Int]](s.serialize(a)) + assert(aSer.tripletIterator().toList === a.tripletIterator().toList) + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/impl/VertexPartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/impl/VertexPartitionSuite.scala new file mode 100644 index 000000000..4b3a8fd73 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/impl/VertexPartitionSuite.scala @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.impl + +import org.apache.spark.SparkConf +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.serializer.KryoSerializer +import org.scalatest.funsuite.AnyFunSuite + +class VertexPartitionSuite extends AnyFunSuite { + + test("isDefined, filter") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1))).filter { (vid, _) => vid == 0 } + assert(vp.isDefined(0)) + assert(!vp.isDefined(1)) + assert(!vp.isDefined(2)) + assert(!vp.isDefined(-1)) + } + + test("map") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1))).map { (_, _) => 2 } + assert(vp(0) === 2) + } + + test("diff") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val vp2 = vp.filter { (vid, _) => vid <= 1 } + val vp3a = vp.map { (_, _) => 2 } + val vp3b = VertexPartition(vp3a.iterator) + // diff with same index + val diff1 = vp2.diff(vp3a) + assert(diff1(0) === 2) + assert(diff1(1) === 2) + assert(diff1(2) === 2) + assert(!diff1.isDefined(2)) + // diff with different indexes + val diff2 = vp2.diff(vp3b) + assert(diff2(0) === 2) + assert(diff2(1) === 2) + assert(diff2(2) === 2) + assert(!diff2.isDefined(2)) + } + + test("leftJoin") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val vp2a = vp.filter { (vid, _) => vid <= 1 }.map { (_, _) => 2 } + val vp2b = VertexPartition(vp2a.iterator) + // leftJoin with same index + val join1 = vp.leftJoin(vp2a) { (_, a, bOpt) => bOpt.getOrElse(a) } + assert(join1.iterator.toSet === Set((0L, 2), (1L, 2), (2L, 1))) + // leftJoin with different indexes + val join2 = vp.leftJoin(vp2b) { (_, a, bOpt) => bOpt.getOrElse(a) } + assert(join2.iterator.toSet === Set((0L, 2), (1L, 2), (2L, 1))) + // leftJoin an iterator + val join3 = vp.leftJoin(vp2a.iterator) { (_, a, bOpt) => bOpt.getOrElse(a) } + assert(join3.iterator.toSet === Set((0L, 2), (1L, 2), (2L, 1))) + } + + test("innerJoin") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val vp2a = vp.filter { (vid, _) => vid <= 1 }.map { (_, _) => 2 } + val vp2b = VertexPartition(vp2a.iterator) + // innerJoin with same index + val join1 = vp.innerJoin(vp2a) { (_, _, b) => b } + assert(join1.iterator.toSet === Set((0L, 2), (1L, 2))) + // innerJoin with different indexes + val join2 = vp.innerJoin(vp2b) { (_, _, b) => b } + assert(join2.iterator.toSet === Set((0L, 2), (1L, 2))) + // innerJoin an iterator + val join3 = vp.innerJoin(vp2a.iterator) { (_, _, b) => b } + assert(join3.iterator.toSet === Set((0L, 2), (1L, 2))) + } + + test("createUsingIndex") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val elems = List((0L, 2), (2L, 2), (3L, 2)) + val vp2 = vp.createUsingIndex(elems.iterator) + assert(vp2.iterator.toSet === Set((0L, 2), (2L, 2))) + assert(vp.index === vp2.index) + } + + test("innerJoinKeepLeft") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val elems = List((0L, 2), (2L, 2), (3L, 2)) + val vp2 = vp.innerJoinKeepLeft(elems.iterator) + assert(vp2.iterator.toSet === Set((0L, 2), (2L, 2))) + assert(vp2(1) === 1) + } + + test("aggregateUsingIndex") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val messages = List((0L, "a"), (2L, "b"), (0L, "c"), (3L, "d")) + val vp2 = vp.aggregateUsingIndex[String](messages.iterator, _ + _) + assert(vp2.iterator.toSet === Set((0L, "ac"), (2L, "b"))) + } + + test("reindex") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val vp2 = vp.filter { (vid, _) => vid <= 1 } + val vp3 = vp2.reindex() + assert(vp2.iterator.toSet === vp3.iterator.toSet) + assert(vp2(2) === 1) + assert(vp3.index.getPos(2) === -1) + } + + test("serialization") { + val verts = Set((0L, 1), (1L, 1), (2L, 1)) + val vp = VertexPartition(verts.iterator) + val javaSer = new JavaSerializer(new SparkConf()) + val conf = new SparkConf() + GraphXUtils.registerKryoClasses(conf) + val kryoSer = new KryoSerializer(conf) + + for (ser <- List(javaSer, kryoSer); s = ser.newInstance()) { + val vpSer: VertexPartition[Int] = s.deserialize[VertexPartition[Int]](s.serialize(vp)) + assert(vpSer.iterator.toSet === verts) + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/ConnectedComponentsSuite.scala new file mode 100644 index 000000000..ce567ba14 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/ConnectedComponentsSuite.scala @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.lib + +import org.apache.spark.graphframes.graphx.LocalSparkContext +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.graphframes.graphx.util.GraphGenerators +import org.apache.spark.rdd._ +import org.scalatest.funsuite.AnyFunSuite + +class ConnectedComponentsSuite extends AnyFunSuite with LocalSparkContext { + + test("Grid Connected Components") { + withSpark { sc => + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10) + val ccGraph = gridGraph.connectedComponents() + val maxCCid = ccGraph.vertices.map { case (_, ccId) => ccId }.sum() + assert(maxCCid === 0) + } + } // end of Grid connected components + + test("Reverse Grid Connected Components") { + withSpark { sc => + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse + val ccGraph = gridGraph.connectedComponents() + val maxCCid = ccGraph.vertices.map { case (_, ccId) => ccId }.sum() + assert(maxCCid === 0) + } + } // end of Grid connected components + + test("Chain Connected Components") { + withSpark { sc => + val chain1 = (0 until 9).map(x => (x, x + 1)) + val chain2 = (10 until 20).map(x => (x, x + 1)) + val rawEdges = + sc.parallelize(chain1 ++ chain2, 3).map { case (s, d) => (s.toLong, d.toLong) } + val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0) + val ccGraph = twoChains.connectedComponents() + val vertices = ccGraph.vertices.collect() + for ((id, cc) <- vertices) { + if (id < 10) { + assert(cc === 0) + } else { + assert(cc === 10) + } + } + val ccMap = vertices.toMap + for (id <- 0 until 20) { + if (id < 10) { + assert(ccMap(id) === 0) + } else { + assert(ccMap(id) === 10) + } + } + } + } // end of chain connected components + + test("Reverse Chain Connected Components") { + withSpark { sc => + val chain1 = (0 until 9).map(x => (x, x + 1)) + val chain2 = (10 until 20).map(x => (x, x + 1)) + val rawEdges = + sc.parallelize(chain1 ++ chain2, 3).map { case (s, d) => (s.toLong, d.toLong) } + val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse + val ccGraph = twoChains.connectedComponents() + val vertices = ccGraph.vertices.collect() + for ((id, cc) <- vertices) { + if (id < 10) { + assert(cc === 0) + } else { + assert(cc === 10) + } + } + val ccMap = vertices.toMap + for (id <- 0 until 20) { + if (id < 10) { + assert(ccMap(id) === 0) + } else { + assert(ccMap(id) === 10) + } + } + } + } // end of reverse chain connected components + + test("Connected Components on a Toy Connected Graph") { + withSpark { sc => + // Create an RDD for the vertices + val users: RDD[(VertexId, (String, String))] = + sc.parallelize( + Seq( + (3L, ("rxin", "student")), + (7L, ("jgonzal", "postdoc")), + (5L, ("franklin", "prof")), + (2L, ("istoica", "prof")), + (4L, ("peter", "student")))) + // Create an RDD for edges + val relationships: RDD[Edge[String]] = + sc.parallelize( + Seq( + Edge(3L, 7L, "collab"), + Edge(5L, 3L, "advisor"), + Edge(2L, 5L, "colleague"), + Edge(5L, 7L, "pi"), + Edge(4L, 0L, "student"), + Edge(5L, 0L, "colleague"))) + // Edges are: + // 2 ---> 5 ---> 3 + // | \ + // V \| + // 4 ---> 0 7 + // + // Define a default user in case there are relationship with missing user + val defaultUser = ("John Doe", "Missing") + // Build the initial Graph + val graph = Graph(users, relationships, defaultUser) + val ccGraph = graph.connectedComponents() + val vertices = ccGraph.vertices.collect() + for ((_, cc) <- vertices) { + assert(cc === 0) + } + } + } // end of toy connected components + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/LabelPropagationSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/LabelPropagationSuite.scala new file mode 100644 index 000000000..84f276443 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/LabelPropagationSuite.scala @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.lib + +import org.apache.spark.graphframes.graphx.LocalSparkContext +import org.apache.spark.graphframes.graphx._ +import org.scalatest.funsuite.AnyFunSuite + +class LabelPropagationSuite extends AnyFunSuite with LocalSparkContext { + test("Label Propagation") { + withSpark { sc => + // Construct a graph with two cliques connected by a single edge + val n = 5 + val clique1 = for (u <- 0L until n; v <- 0L until n) yield Edge(u, v, 1) + val clique2 = for (u <- 0L to n; v <- 0L to n) yield Edge(u + n, v + n, 1) + val twoCliques = sc.parallelize(clique1 ++ clique2 :+ Edge(0L, n, 1)) + val graph = Graph.fromEdges(twoCliques, 1) + // Run label propagation + val labels = LabelPropagation.run(graph, n * 4).cache() + + // All vertices within a clique should have the same label + val clique1Labels = labels.vertices.filter(_._1 < n).map(_._2).collect().toArray + assert(clique1Labels.forall(_ == clique1Labels(0))) + val clique2Labels = labels.vertices.filter(_._1 >= n).map(_._2).collect().toArray + assert(clique2Labels.forall(_ == clique2Labels(0))) + // The two cliques should have different labels + assert(clique1Labels(0) != clique2Labels(0)) + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/PageRankSuite.scala new file mode 100644 index 000000000..116bc1003 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/PageRankSuite.scala @@ -0,0 +1,484 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.lib + +import org.apache.spark.graphframes.graphx.LocalSparkContext +import org.apache.spark.graphframes.graphx._ +import org.apache.spark.graphframes.graphx.util.GraphGenerators +import org.scalatest.funsuite.AnyFunSuite + +object GridPageRank { + def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double): Seq[(VertexId, Double)] = { + val inNbrs = Array.fill(nRows * nCols)(collection.mutable.ArrayBuffer.empty[Int]) + val outDegree = Array.fill(nRows * nCols)(0) + // Convert row column address into vertex ids (row major order) + def sub2ind(r: Int, c: Int): Int = r * nCols + c + // Make the grid graph + for (r <- 0 until nRows; c <- 0 until nCols) { + val ind = sub2ind(r, c) + if (r + 1 < nRows) { + outDegree(ind) += 1 + inNbrs(sub2ind(r + 1, c)) += ind + } + if (c + 1 < nCols) { + outDegree(ind) += 1 + inNbrs(sub2ind(r, c + 1)) += ind + } + } + // compute the pagerank + var pr = Array.fill(nRows * nCols)(1.0) + for (_ <- 0 until nIter) { + val oldPr = pr + pr = new Array[Double](nRows * nCols) + for (ind <- 0 until (nRows * nCols)) { + pr(ind) = resetProb + (1.0 - resetProb) * + inNbrs(ind).map(nbr => oldPr(nbr) / outDegree(nbr)).sum + } + } + val prSum = pr.sum + (0L until (nRows * nCols)).zip(pr.map(_ * pr.length / prSum)) + } + +} + +class PageRankSuite extends AnyFunSuite with LocalSparkContext { + + def compareRanks(a: VertexRDD[Double], b: VertexRDD[Double]): Double = { + a.leftJoin(b) { case (_, a, bOpt) => (a - bOpt.getOrElse(0.0)) * (a - bOpt.getOrElse(0.0)) } + .map { case (_, error) => error } + .sum() + } + + def convergenceIterations[VD, ED]( + graph: Graph[VD, ED], + resetProb: Double, + tol: Double, + errorTol: Double): (Int, Int) = { + val dynamicRanks = graph.ops.pageRank(tol, resetProb).vertices.cache() + + // Compute how many iterations it takes to converge + var iter = 1 + var staticGraphRank = graph.ops.staticPageRank(iter, resetProb).vertices.cache() + while (!(compareRanks(staticGraphRank, dynamicRanks) < errorTol)) { + iter += 1 + staticGraphRank = graph.ops.staticPageRank(iter, resetProb).vertices.cache() + } + val convergenceIter = iter + val checkPointIter = convergenceIter / 2 + + // CheckPoint the graph computed at half of these iterations + val staticGraphRankPartial = graph.ops.staticPageRank(checkPointIter, resetProb) + + // Compute how many iterations it takes to converge when a checkPoint is provided + var iterWithCheckPoint = 1 + var staticGraphRankWithCheckPoint = graph.ops + .staticPageRank(iterWithCheckPoint, resetProb, staticGraphRankPartial) + .vertices + .cache() + while (compareRanks(staticGraphRankWithCheckPoint, dynamicRanks) >= errorTol) { + iterWithCheckPoint += 1 + staticGraphRankWithCheckPoint = graph.ops + .staticPageRank(iterWithCheckPoint, resetProb, staticGraphRankPartial) + .vertices + .cache() + } + + val convergenceIterWithCheckPoint = iterWithCheckPoint + + (convergenceIterWithCheckPoint, convergenceIter) + } + + test("Star PageRank") { + withSpark { sc => + val nVertices = 100 + val starGraph = GraphGenerators.starGraph(sc, nVertices).cache() + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 2 + val errorTol = 1.0e-5 + + val staticRanks = starGraph.staticPageRank(numIter, resetProb).vertices.cache() + val staticRanks2 = starGraph.staticPageRank(numIter + 1, resetProb).vertices + + // Static PageRank should only take 2 iterations to converge + val notMatching = staticRanks + .innerZipJoin(staticRanks2) { (_, pr1, pr2) => + if (pr1 != pr2) 1 else 0 + } + .map { case (_, test) => test } + .sum() + assert(notMatching === 0) + + val dynamicRanks = starGraph.pageRank(tol, resetProb).vertices.cache() + assert(compareRanks(staticRanks, dynamicRanks) < errorTol) + + // Computed in igraph 1.0 w/ R bindings: + // > page_rank(make_star(100, mode = "in")) + // Alternatively in NetworkX 1.11: + // > nx.pagerank(nx.DiGraph([(x, 0) for x in range(1,100)])) + // We multiply by the number of vertices to account for difference in normalization + val centerRank = 0.462394787 * nVertices + val othersRank = 0.005430356 * nVertices + val igraphPR = centerRank +: Seq.fill(nVertices - 1)(othersRank) + val ranks = VertexRDD(sc.parallelize(0L until nVertices zip igraphPR)) + assert(compareRanks(staticRanks, ranks) < errorTol) + assert(compareRanks(dynamicRanks, ranks) < errorTol) + + } + } // end of test Star PageRank + + test("Star PersonalPageRank") { + withSpark { sc => + val nVertices = 100 + val starGraph = GraphGenerators.starGraph(sc, nVertices).cache() + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 2 + val errorTol = 1.0e-5 + + val staticRanks = + starGraph.staticPersonalizedPageRank(0, numIter, resetProb).vertices.cache() + + val dynamicRanks = starGraph.personalizedPageRank(0, tol, resetProb).vertices.cache() + assert(compareRanks(staticRanks, dynamicRanks) < errorTol) + + val parallelStaticRanks = starGraph + .staticParallelPersonalizedPageRank(Array(0), numIter, resetProb) + .mapVertices { case (_, vector) => + vector(0) + } + .vertices + .cache() + assert(compareRanks(staticRanks, parallelStaticRanks) < errorTol) + + // Computed in igraph 1.0 w/ R bindings: + // > page_rank(make_star(100, mode = "in"), personalized = c(1, rep(0, 99)), algo = "arpack") + // NOTE: We use the arpack algorithm as prpack (the default) redistributes rank to all + // vertices uniformly instead of just to the personalization source. + // Alternatively in NetworkX 1.11: + // > nx.pagerank(nx.DiGraph([(x, 0) for x in range(1,100)]), + // personalization=dict([(x, 1 if x == 0 else 0) for x in range(0,100)])) + // We multiply by the number of vertices to account for difference in normalization + val igraphPR0 = 1.0 +: Seq.fill(nVertices - 1)(0.0) + val ranks0 = VertexRDD(sc.parallelize(0L until nVertices zip igraphPR0)) + assert(compareRanks(staticRanks, ranks0) < errorTol) + assert(compareRanks(dynamicRanks, ranks0) < errorTol) + + // We have one outbound edge from 1 to 0 + val otherStaticRanks = + starGraph.staticPersonalizedPageRank(1, numIter, resetProb).vertices.cache() + val otherDynamicRanks = starGraph.personalizedPageRank(1, tol, resetProb).vertices.cache() + val otherParallelStaticRanks = starGraph + .staticParallelPersonalizedPageRank(Array(0, 1), numIter, resetProb) + .mapVertices { case (_, vector) => + vector(1) + } + .vertices + .cache() + assert(compareRanks(otherDynamicRanks, otherStaticRanks) < errorTol) + assert(compareRanks(otherStaticRanks, otherParallelStaticRanks) < errorTol) + assert(compareRanks(otherDynamicRanks, otherParallelStaticRanks) < errorTol) + + // Computed in igraph 1.0 w/ R bindings: + // > page_rank(make_star(100, mode = "in"), + // personalized = c(0, 1, rep(0, 98)), algo = "arpack") + // NOTE: We use the arpack algorithm as prpack (the default) redistributes rank to all + // vertices uniformly instead of just to the personalization source. + // Alternatively in NetworkX 1.11: + // > nx.pagerank(nx.DiGraph([(x, 0) for x in range(1,100)]), + // personalization=dict([(x, 1 if x == 1 else 0) for x in range(0,100)])) + val centerRank = 0.4594595 + val sourceRank = 0.5405405 + val igraphPR1 = centerRank +: sourceRank +: Seq.fill(nVertices - 2)(0.0) + val ranks1 = VertexRDD(sc.parallelize(0L until nVertices zip igraphPR1)) + assert(compareRanks(otherStaticRanks, ranks1) < errorTol) + assert(compareRanks(otherDynamicRanks, ranks1) < errorTol) + assert(compareRanks(otherParallelStaticRanks, ranks1) < errorTol) + } + } // end of test Star PersonalPageRank + + test("Grid PageRank") { + withSpark { sc => + val rows = 10 + val cols = 10 + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 50 + val errorTol = 1.0e-5 + val gridGraph = GraphGenerators.gridGraph(sc, rows, cols).cache() + + val staticRanks = gridGraph.staticPageRank(numIter, resetProb).vertices.cache() + val dynamicRanks = gridGraph.pageRank(tol, resetProb).vertices.cache() + val referenceRanks = + VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))).cache() + + assert(compareRanks(staticRanks, referenceRanks) < errorTol) + assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) + } + } // end of Grid PageRank + + test("Grid PageRank with checkpoint") { + withSpark { sc => + // Check that checkPointing helps the static PageRank to converge in less iterations + val rows = 10 + val cols = 10 + val resetProb = 0.15 + val errorTol = 1.0e-5 + val tol = 0.0001 + val gridGraph = GraphGenerators.gridGraph(sc, rows, cols).cache() + + val (iterAfterHalfCheckPoint, totalIters) = + convergenceIterations(gridGraph, resetProb, tol, errorTol) + + // In this case checkPoint does not help much + assert(totalIters == 19) + assert(iterAfterHalfCheckPoint == 18) + } + } // end of Grid PageRank with checkpoint + + test("Grid PageRank with checkpoint without intermediate normalization") { + withSpark { sc => + // Check that 6 iterations in a row are equivalent + // to 3 times 2 iterations without intermediate normalization + val rows = 10 + val cols = 10 + val resetProb = 0.15 + val gridGraph = GraphGenerators.gridGraph(sc, rows, cols).cache() + + val ranksA: Array[(VertexId, Double)] = PageRank + .runWithOptions(gridGraph, numIter = 6, resetProb, srcId = None, normalized = true) + .vertices + .collect() + + val preRankGraph1 = PageRank.runWithOptions( + gridGraph, + numIter = 2, + resetProb, + srcId = None, + normalized = false) + + val preRankGraph2 = PageRank.runWithOptionsWithPreviousPageRank( + gridGraph, + numIter = 2, + resetProb, + srcId = None, + normalized = false, + preRankGraph1) + + val ranksB: Array[(VertexId, Double)] = PageRank + .runWithOptionsWithPreviousPageRank( + gridGraph, + numIter = 2, + resetProb, + srcId = None, + normalized = true, + preRankGraph2) + .vertices + .collect() + + // assert that all scores are equal + assert(ranksA.zip(ranksB).forall { case (rankA, rankB) => rankA == rankB }) + } + } // end of Grid PageRank with checkpoint without intermediate normalization + + test("Chain PageRank") { + withSpark { sc => + val chain1 = (0 until 9).map(x => (x, x + 1)) + val rawEdges = sc.parallelize(chain1, 1).map { case (s, d) => (s.toLong, d.toLong) } + val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 10 + val errorTol = 1.0e-5 + + val staticRanks = chain.staticPageRank(numIter, resetProb).vertices + val dynamicRanks = chain.pageRank(tol, resetProb).vertices + + assert(compareRanks(staticRanks, dynamicRanks) < errorTol) + } + } + + test("Chain PageRank with checkpoint") { + withSpark { sc => + val chain1 = (0 until 9).map(x => (x, x + 1)) + val rawEdges = sc.parallelize(chain1, 1).map { case (s, d) => (s.toLong, d.toLong) } + val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val resetProb = 0.15 + val errorTol = 1.0e-5 + val tol = 0.0001 + + val (iterAfterHalfCheckPoint, totalIters) = + convergenceIterations(chain, resetProb, tol, errorTol) + + // In this case checkPoint does not help but it does not take more iterations + assert(totalIters == 10) + assert(iterAfterHalfCheckPoint == 10) + } + } // end of Grid PageRank + + test("Chain PersonalizedPageRank") { + withSpark { sc => + // Check that implementation can handle large vertexIds, SPARK-25149 + val vertexIdOffset = Int.MaxValue.toLong + 1 + val sourceOffset = 4 + val source = vertexIdOffset + sourceOffset + val numIter = 10 + val vertices = vertexIdOffset until vertexIdOffset + numIter + val chain1 = vertices.zip(vertices.tail) + val rawEdges = sc.parallelize(chain1, 1).map { case (s, d) => (s.toLong, d.toLong) } + val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val resetProb = 0.15 + val tol = 0.0001 + val errorTol = 1.0e-1 + + val a = resetProb / (1 - Math.pow(1 - resetProb, numIter - sourceOffset)) + // We expect the rank to decay as (1 - resetProb) ^ distance + val expectedRanks = sc.parallelize(vertices).map { vid => + val rank = if (vid < source) { + 0.0 + } else { + a * Math.pow(1 - resetProb, vid.toDouble - source) + } + vid -> rank + } + val expected = VertexRDD(expectedRanks) + + val staticRanks = chain.staticPersonalizedPageRank(source, numIter, resetProb).vertices + assert(compareRanks(staticRanks, expected) < errorTol) + + val dynamicRanks = chain.personalizedPageRank(source, tol, resetProb).vertices + assert(compareRanks(dynamicRanks, expected) < errorTol) + + val parallelStaticRanks = chain + .staticParallelPersonalizedPageRank(Array(source), numIter, resetProb) + .mapVertices { case (_, vector) => + vector(0) + } + .vertices + .cache() + assert(compareRanks(parallelStaticRanks, expected) < errorTol) + } + } + + test("Loop with source PageRank") { + withSpark { sc => + val edges = sc.parallelize((1L, 2L) :: (2L, 3L) :: (3L, 4L) :: (4L, 2L) :: Nil) + val g = Graph.fromEdgeTuples(edges, 1) + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 50 + val errorTol = 1.0e-5 + + val staticRanks = g.staticPageRank(numIter, resetProb).vertices + val dynamicRanks = g.pageRank(tol, resetProb).vertices + assert(compareRanks(staticRanks, dynamicRanks) < errorTol) + + // Computed in igraph 1.0 w/ R bindings: + // > page_rank(graph_from_literal( A -+ B -+ C -+ D -+ B)) + // Alternatively in NetworkX 1.11: + // > nx.pagerank(nx.DiGraph([(1,2),(2,3),(3,4),(4,2)])) + // We multiply by the number of vertices to account for difference in normalization + val igraphPR = Seq(0.0375000, 0.3326045, 0.3202138, 0.3096817).map(_ * 4) + val ranks = VertexRDD(sc.parallelize(1L to 4L zip igraphPR)) + assert(compareRanks(staticRanks, ranks) < errorTol) + assert(compareRanks(dynamicRanks, ranks) < errorTol) + } + } + + test("Loop with source PageRank with checkpoint") { + withSpark { sc => + val edges = sc.parallelize((1L, 2L) :: (2L, 3L) :: (3L, 4L) :: (4L, 2L) :: Nil) + val g = Graph.fromEdgeTuples(edges, 1) + val resetProb = 0.15 + val tol = 0.0001 + val errorTol = 1.0e-5 + + val (iterAfterHalfCheckPoint, totalIters) = + convergenceIterations(g, resetProb, tol, errorTol) + + // In this case checkPoint helps a lot + assert(totalIters == 34) + assert(iterAfterHalfCheckPoint == 17) + } + } + + test("Loop with sink PageRank") { + withSpark { sc => + val edges = sc.parallelize((1L, 2L) :: (2L, 3L) :: (3L, 1L) :: (1L, 4L) :: Nil) + val g = Graph.fromEdgeTuples(edges, 1) + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 20 + val errorTol = 1.0e-5 + + val staticRanks = g.staticPageRank(numIter, resetProb).vertices.cache() + val dynamicRanks = g.pageRank(tol, resetProb).vertices.cache() + + assert(compareRanks(staticRanks, dynamicRanks) < errorTol) + + // Computed in igraph 1.0 w/ R bindings: + // > page_rank(graph_from_literal( A -+ B -+ C -+ A -+ D)) + // Alternatively in NetworkX 1.11: + // > nx.pagerank(nx.DiGraph([(1,2),(2,3),(3,1),(1,4)])) + // We multiply by the number of vertices to account for difference in normalization + val igraphPR = Seq(0.3078534, 0.2137622, 0.2646223, 0.2137622).map(_ * 4) + val ranks = VertexRDD(sc.parallelize(1L to 4L zip igraphPR)) + assert(compareRanks(staticRanks, ranks) < errorTol) + assert(compareRanks(dynamicRanks, ranks) < errorTol) + + val p1staticRanks = g.staticPersonalizedPageRank(1, numIter, resetProb).vertices.cache() + val p1dynamicRanks = g.personalizedPageRank(1, tol, resetProb).vertices.cache() + val p1parallelDynamicRanks = + g.staticParallelPersonalizedPageRank(Array(1, 2, 3, 4), numIter, resetProb) + .vertices + .mapValues(v => v(0)) + .cache() + + // Computed in igraph 1.0 w/ R bindings: + // > page_rank(graph_from_literal( A -+ B -+ C -+ A -+ D), personalized = c(1, 0, 0, 0), + // algo = "arpack") + // NOTE: We use the arpack algorithm as prpack (the default) redistributes rank to all + // vertices uniformly instead of just to the personalization source. + // Alternatively in NetworkX 1.11: + // > nx.pagerank(nx.DiGraph([(1,2),(2,3),(3,1),(1,4)]), personalization={1:1, 2:0, 3:0, 4:0}) + val igraphPR2 = Seq(0.4522329, 0.1921990, 0.1633691, 0.1921990) + val ranks2 = VertexRDD(sc.parallelize(1L to 4L zip igraphPR2)) + assert(compareRanks(p1staticRanks, ranks2) < errorTol) + assert(compareRanks(p1dynamicRanks, ranks2) < errorTol) + assert(compareRanks(p1parallelDynamicRanks, ranks2) < errorTol) + + } + } + + test("Loop with sink PageRank with checkpoint") { + withSpark { sc => + val edges = sc.parallelize((1L, 2L) :: (2L, 3L) :: (3L, 1L) :: (1L, 4L) :: Nil) + val g = Graph.fromEdgeTuples(edges, 1) + val resetProb = 0.15 + val tol = 0.0001 + val errorTol = 1.0e-5 + + val (iterAfterHalfCheckPoint, totalIters) = + convergenceIterations(g, resetProb, tol, errorTol) + + // In this case checkPoint helps a lot + assert(totalIters == 15) + assert(iterAfterHalfCheckPoint == 9) + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/SVDPlusPlusSuite.scala new file mode 100644 index 000000000..e0573edc6 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/SVDPlusPlusSuite.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.lib + +import org.apache.spark.graphframes.graphx.LocalSparkContext +import org.apache.spark.graphframes.graphx._ +import org.scalatest.funsuite.AnyFunSuite + +class SVDPlusPlusSuite extends AnyFunSuite with LocalSparkContext { + + test("Test SVD++ with mean square error on training set") { + withSpark { sc => + val svdppErr = 8.0 + val edges = sc + .parallelize(Seq( + Seq(1, 1, 5.0), + Seq(1, 2, 1.0), + Seq(1, 3, 5.0), + Seq(1, 4, 1.0), + Seq(2, 1, 5.0), + Seq(2, 2, 1.0), + Seq(2, 3, 5.0), + Seq(2, 4, 1.0), + Seq(3, 1, 1.0), + Seq(3, 2, 5.0), + Seq(3, 3, 1.0), + Seq(3, 4, 5.0), + Seq(4, 1, 1.0), + Seq(4, 2, 5.0), + Seq(4, 3, 1.0), + Seq(4, 4, 5.0))) + .map(f => Edge(f(0).toLong * 2, f(1).toLong * 2 + 1, f(2))) + val conf = new SVDPlusPlus.Conf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations + val (graph, _) = SVDPlusPlus.run(edges, conf) + graph.cache() + val err = graph.vertices + .map { case (vid, vd) => + if (vid % 2 == 1) vd._4 else 0.0 + } + .reduce(_ + _) / graph.numEdges + assert(err <= svdppErr) + } + } + + test("Test SVD++ with no edges") { + withSpark { sc => + val edges = sc.emptyRDD[Edge[Double]] + val conf = new SVDPlusPlus.Conf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations + val (graph, _) = SVDPlusPlus.run(edges, conf) + assert(graph.vertices.count() == 0) + assert(graph.edges.count() == 0) + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/ShortestPathsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/ShortestPathsSuite.scala new file mode 100644 index 000000000..5a1090142 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/ShortestPathsSuite.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.lib + +import org.apache.spark.graphframes.graphx.LocalSparkContext +import org.apache.spark.graphframes.graphx._ +import org.scalatest.funsuite.AnyFunSuite + +class ShortestPathsSuite extends AnyFunSuite with LocalSparkContext { + + test("Shortest Path Computations") { + withSpark { sc => + val shortestPaths = Set( + (1, Map(1 -> 0, 4 -> 2)), + (2, Map(1 -> 1, 4 -> 2)), + (3, Map(1 -> 2, 4 -> 1)), + (4, Map(1 -> 2, 4 -> 0)), + (5, Map(1 -> 1, 4 -> 1)), + (6, Map(1 -> 3, 4 -> 1))) + val edgeSeq = Seq((1, 2), (1, 5), (2, 3), (2, 5), (3, 4), (4, 5), (4, 6)).flatMap { + case e => Seq(e, e.swap) + } + val edges = sc.parallelize(edgeSeq).map { case (v1, v2) => (v1.toLong, v2.toLong) } + val graph = Graph.fromEdgeTuples(edges, 1) + val landmarks = Seq(1, 4).map(_.toLong) + val results = ShortestPaths.run(graph, landmarks).vertices.collect().map { + case (v, spMap) => (v, spMap.toMap.transform((_, i) => i)) + } + assert(results.toSet === shortestPaths) + } + } + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/StronglyConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/StronglyConnectedComponentsSuite.scala new file mode 100644 index 000000000..12530e216 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/StronglyConnectedComponentsSuite.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.lib + +import org.apache.spark.graphframes.graphx.LocalSparkContext +import org.apache.spark.graphframes.graphx._ +import org.scalatest.funsuite.AnyFunSuite + +class StronglyConnectedComponentsSuite extends AnyFunSuite with LocalSparkContext { + + test("Island Strongly Connected Components") { + withSpark { sc => + val vertices = sc.parallelize((1L to 5L).map(x => (x, -1))) + val edges = sc.parallelize(Seq.empty[Edge[Int]]) + val graph = Graph[Int, Int](vertices, edges) + val sccGraph = graph.stronglyConnectedComponents(5) + for ((id, scc) <- sccGraph.vertices.collect()) { + assert(id === scc) + } + } + } + + test("Cycle Strongly Connected Components") { + withSpark { sc => + val rawEdges = sc.parallelize((0L to 6L).map(x => (x, (x + 1) % 7))) + val graph = Graph.fromEdgeTuples(rawEdges, -1) + val sccGraph = graph.stronglyConnectedComponents(20) + for ((_, scc) <- sccGraph.vertices.collect()) { + assert(0L === scc) + } + } + } + + test("2 Cycle Strongly Connected Components") { + withSpark { sc => + val edges = + Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(3L -> 4L, 4L -> 5L, 5L -> 3L) ++ + Array(6L -> 0L, 5L -> 7L) + val rawEdges = sc.parallelize(edges.toIndexedSeq) + val graph = Graph.fromEdgeTuples(rawEdges, -1) + val sccGraph = graph.stronglyConnectedComponents(20) + for ((id, scc) <- sccGraph.vertices.collect()) { + if (id < 3) { + assert(0L === scc) + } else if (id < 6) { + assert(3L === scc) + } else { + assert(id === scc) + } + } + } + } + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/TriangleCountSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/TriangleCountSuite.scala new file mode 100644 index 000000000..5de15363e --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/lib/TriangleCountSuite.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.lib + +import org.apache.spark.graphframes.graphx.LocalSparkContext +import org.apache.spark.graphframes.graphx.PartitionStrategy.RandomVertexCut +import org.apache.spark.graphframes.graphx._ +import org.scalatest.funsuite.AnyFunSuite + +class TriangleCountSuite extends AnyFunSuite with LocalSparkContext { + + test("Count a single triangle") { + withSpark { sc => + val rawEdges = sc.parallelize(Seq(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) + val graph = Graph.fromEdgeTuples(rawEdges, true).cache() + val triangleCount = graph.triangleCount() + val verts = triangleCount.vertices + verts.collect().foreach { case (_, count) => assert(count === 1) } + } + } + + test("Count two triangles") { + withSpark { sc => + val triangles = Seq(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Seq(0L -> -1L, -1L -> -2L, -2L -> 0L) + val rawEdges = sc.parallelize(triangles, 2) + val graph = Graph.fromEdgeTuples(rawEdges, true).cache() + val triangleCount = graph.triangleCount() + val verts = triangleCount.vertices + verts.collect().foreach { case (vid, count) => + if (vid == 0) { + assert(count === 2) + } else { + assert(count === 1) + } + } + } + } + + test("Count two triangles with bi-directed edges") { + withSpark { sc => + val triangles = + Seq(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Seq(0L -> -1L, -1L -> -2L, -2L -> 0L) + val revTriangles = triangles.map { case (a, b) => (b, a) } + val rawEdges = sc.parallelize(triangles ++ revTriangles, 2) + val graph = Graph.fromEdgeTuples(rawEdges, true).cache() + val triangleCount = graph.triangleCount() + val verts = triangleCount.vertices + verts.collect().foreach { case (vid, count) => + if (vid == 0) { + assert(count === 2) + } else { + assert(count === 1) + } + } + } + } + + test("Count a single triangle with duplicate edges") { + withSpark { sc => + val rawEdges = sc.parallelize( + Seq(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Seq(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Seq(1L -> 0L, 1L -> 1L), + 2) + val graph = + Graph.fromEdgeTuples(rawEdges, true, uniqueEdges = Some(RandomVertexCut)).cache() + val triangleCount = graph.triangleCount() + val verts = triangleCount.vertices + verts.collect().foreach { case (_, count) => assert(count === 1) } + } + } + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/util/GraphGeneratorsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/util/GraphGeneratorsSuite.scala new file mode 100644 index 000000000..26923ee55 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/util/GraphGeneratorsSuite.scala @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.util + +import org.apache.spark.graphframes.graphx.LocalSparkContext +import org.scalatest.funsuite.AnyFunSuite + +class GraphGeneratorsSuite extends AnyFunSuite with LocalSparkContext { + + test("GraphGenerators.generateRandomEdges") { + val src = 5 + val numEdges10 = 10 + val numEdges20 = 20 + val maxVertexId = 100 + + val edges10 = GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId) + assert(edges10.length == numEdges10) + + val correctSrc = edges10.forall(e => e.srcId == src) + assert(correctSrc) + + val correctWeight = edges10.forall(e => e.attr == 1) + assert(correctWeight) + + val correctRange = edges10.forall(e => e.dstId >= 0 && e.dstId <= maxVertexId) + assert(correctRange) + + val edges20 = GraphGenerators.generateRandomEdges(src, numEdges20, maxVertexId) + assert(edges20.length == numEdges20) + + val edges10_round1 = + GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed = 12345) + val edges10_round2 = + GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed = 12345) + assert(edges10_round1.zip(edges10_round2).forall { case (e1, e2) => + e1.srcId == e2.srcId && e1.dstId == e2.dstId && e1.attr == e2.attr + }) + + val edges10_round3 = + GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed = 3467) + assert(!edges10_round1.zip(edges10_round3).forall { case (e1, e2) => + e1.srcId == e2.srcId && e1.dstId == e2.dstId && e1.attr == e2.attr + }) + } + + test("GraphGenerators.sampleLogNormal") { + val mu = 4.0 + val sigma = 1.3 + val maxVal = 100 + + val trials = 1000 + for (_ <- 1 to trials) { + val dstId = GraphGenerators.sampleLogNormal(mu, sigma, maxVal) + assert(dstId < maxVal) + } + + val dstId_round1 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 12345) + val dstId_round2 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 12345) + assert(dstId_round1 == dstId_round2) + + val dstId_round3 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 789) + assert(dstId_round1 != dstId_round3) + } + + test("GraphGenerators.logNormalGraph") { + withSpark { sc => + val mu = 4.0 + val sigma = 1.3 + val numVertices100 = 100 + + val graph = GraphGenerators.logNormalGraph(sc, numVertices100, mu = mu, sigma = sigma) + assert(graph.vertices.count() == numVertices100) + + val graph_round1 = + GraphGenerators.logNormalGraph(sc, numVertices100, mu = mu, sigma = sigma, seed = 12345) + val graph_round2 = + GraphGenerators.logNormalGraph(sc, numVertices100, mu = mu, sigma = sigma, seed = 12345) + + val graph_round1_edges = graph_round1.edges.collect() + val graph_round2_edges = graph_round2.edges.collect() + + assert(graph_round1_edges.zip(graph_round2_edges).forall { case (e1, e2) => + e1.srcId == e2.srcId && e1.dstId == e2.dstId && e1.attr == e2.attr + }) + + val graph_round3 = + GraphGenerators.logNormalGraph(sc, numVertices100, mu = mu, sigma = sigma, seed = 567) + + val graph_round3_edges = graph_round3.edges.collect() + + assert(!graph_round1_edges.zip(graph_round3_edges).forall { case (e1, e2) => + e1.srcId == e2.srcId && e1.dstId == e2.dstId && e1.attr == e2.attr + }) + } + } + + test("SPARK-5064 GraphGenerators.rmatGraph numEdges upper bound") { + withSpark { sc => + val g1 = GraphGenerators.rmatGraph(sc, 4, 4) + assert(g1.edges.count() === 4) + intercept[IllegalArgumentException] { + GraphGenerators.rmatGraph(sc, 4, 8) + } + } + } + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphframes/graphx/util/PeriodicGraphCheckpointerSuite.scala b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/util/PeriodicGraphCheckpointerSuite.scala new file mode 100644 index 000000000..0eebd2bae --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphframes/graphx/util/PeriodicGraphCheckpointerSuite.scala @@ -0,0 +1,212 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphframes.graphx.util + +import org.apache.hadoop.fs.Path +import org.apache.spark.SparkContext +import org.apache.spark.graphframes.graphx.Edge +import org.apache.spark.graphframes.graphx.Graph +import org.apache.spark.graphframes.graphx.LocalSparkContext +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils +import org.scalatest.Assertions +import org.scalatest.funsuite.AnyFunSuite + +class PeriodicGraphCheckpointerSuite extends AnyFunSuite with LocalSparkContext { + + import PeriodicGraphCheckpointerSuite.* + + test("Persisting") { + var graphsToCheck = Seq.empty[GraphToCheck] + + withSpark { sc => + val graph1 = createGraph(sc) + val checkpointer = + new PeriodicGraphCheckpointer[Double, Double](10, graph1.vertices.sparkContext) + checkpointer.update(graph1) + graphsToCheck = graphsToCheck :+ GraphToCheck(graph1, 1) + checkPersistence(graphsToCheck, 1) + + var iteration = 2 + while (iteration < 9) { + val graph = createGraph(sc) + checkpointer.update(graph) + graphsToCheck = graphsToCheck :+ GraphToCheck(graph, iteration) + checkPersistence(graphsToCheck, iteration) + iteration += 1 + } + } + } + + test("Checkpointing") { + withSpark { sc => + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + val checkpointInterval = 2 + var graphsToCheck = Seq.empty[GraphToCheck] + sc.setCheckpointDir(path) + val graph1 = createGraph(sc) + val checkpointer = new PeriodicGraphCheckpointer[Double, Double]( + checkpointInterval, + graph1.vertices.sparkContext) + checkpointer.update(graph1) + graph1.edges.count() + graph1.vertices.count() + graphsToCheck = graphsToCheck :+ GraphToCheck(graph1, 1) + checkCheckpoint(graphsToCheck, 1, checkpointInterval) + + var iteration = 2 + while (iteration < 9) { + val graph = createGraph(sc) + checkpointer.update(graph) + graph.vertices.count() + graph.edges.count() + graphsToCheck = graphsToCheck :+ GraphToCheck(graph, iteration) + checkCheckpoint(graphsToCheck, iteration, checkpointInterval) + iteration += 1 + } + + checkpointer.deleteAllCheckpoints() + graphsToCheck.foreach { graph => + confirmCheckpointRemoved(graph.graph) + } + + Utils.deleteRecursively(tempDir) + } + } +} + +private object PeriodicGraphCheckpointerSuite extends Assertions { + private val defaultStorageLevel = StorageLevel.MEMORY_ONLY_SER + + case class GraphToCheck(graph: Graph[Double, Double], gIndex: Int) + + val edges: Seq[Edge[Double]] = Seq( + Edge[Double](0, 1, 0), + Edge[Double](1, 2, 0), + Edge[Double](2, 3, 0), + Edge[Double](3, 4, 0)) + + def createGraph(sc: SparkContext): Graph[Double, Double] = { + Graph.fromEdges[Double, Double]( + sc.parallelize(edges), + 0, + defaultStorageLevel, + defaultStorageLevel) + } + + def checkPersistence(graphs: Seq[GraphToCheck], iteration: Int): Unit = { + graphs.foreach { g => + checkPersistence(g.graph, g.gIndex, iteration) + } + } + + /** + * Check storage level of graph. + * @param gIndex + * Index of graph in order inserted into checkpointer (from 1). + * @param iteration + * Total number of graphs inserted into checkpointer. + */ + def checkPersistence(graph: Graph[_, _], gIndex: Int, iteration: Int): Unit = { + try { + if (gIndex + 2 < iteration) { + assert(graph.vertices.getStorageLevel == StorageLevel.NONE) + assert(graph.edges.getStorageLevel == StorageLevel.NONE) + () + } else { + assert(graph.vertices.getStorageLevel == defaultStorageLevel) + assert(graph.edges.getStorageLevel == defaultStorageLevel) + () + } + } catch { + case _: AssertionError => + throw new Exception( + "PeriodicGraphCheckpointerSuite.checkPersistence failed with:\n" + + s"\t gIndex = $gIndex\n" + + s"\t iteration = $iteration\n" + + s"\t graph.vertices.getStorageLevel = ${graph.vertices.getStorageLevel}\n" + + s"\t graph.edges.getStorageLevel = ${graph.edges.getStorageLevel}\n") + } + } + + def checkCheckpoint( + graphs: Seq[GraphToCheck], + iteration: Int, + checkpointInterval: Int): Unit = { + graphs.reverse.foreach { g => + checkCheckpoint(g.graph, g.gIndex, iteration, checkpointInterval) + } + } + + def confirmCheckpointRemoved(graph: Graph[_, _]): Unit = { + // Note: We cannot check graph.isCheckpointed since that value is never updated. + // Instead, we check for the presence of the checkpoint files. + // This test should continue to work even after this graph.isCheckpointed issue + // is fixed (though it can then be simplified and not look for the files). + val hadoopConf = graph.vertices.sparkContext.hadoopConfiguration + graph.getCheckpointFiles.foreach { checkpointFile => + val path = new Path(checkpointFile) + val fs = path.getFileSystem(hadoopConf) + assert(!fs.exists(path), "Graph checkpoint file should have been removed") + } + } + + /** + * Check checkpointed status of graph. + * @param gIndex + * Index of graph in order inserted into checkpointer (from 1). + * @param iteration + * Total number of graphs inserted into checkpointer. + */ + def checkCheckpoint( + graph: Graph[_, _], + gIndex: Int, + iteration: Int, + checkpointInterval: Int): Unit = { + try { + if (gIndex % checkpointInterval == 0) { + // We allow 2 checkpoint intervals since we perform an action (checkpointing a second graph) + // only AFTER PeriodicGraphCheckpointer decides whether to remove the previous checkpoint. + if (iteration - 2 * checkpointInterval < gIndex && gIndex <= iteration) { + assert(graph.isCheckpointed, "Graph should be checkpointed") + assert(graph.getCheckpointFiles.length == 2, "Graph should have 2 checkpoint files") + () + } else { + confirmCheckpointRemoved(graph) + } + } else { + // Graph should never be checkpointed + assert(!graph.isCheckpointed, "Graph should never have been checkpointed") + assert(graph.getCheckpointFiles.isEmpty, "Graph should not have any checkpoint files") + () + } + } catch { + case e: AssertionError => + throw new Exception( + "PeriodicGraphCheckpointerSuite.checkCheckpoint failed with:\n" + + s"\t gIndex = $gIndex\n" + + s"\t iteration = $iteration\n" + + s"\t checkpointInterval = $checkpointInterval\n" + + s"\t graph.isCheckpointed = ${graph.isCheckpointed}\n" + + s"\t graph.getCheckpointFiles = ${graph.getCheckpointFiles.mkString(", ")}\n" + + s" AssertionError message: ${e.getMessage}") + } + } + +} diff --git a/python/README.md b/python/README.md index 093009719..4b8cdb8c7 100644 --- a/python/README.md +++ b/python/README.md @@ -21,6 +21,8 @@ pip install graphframes-py **NOTE!** *Python distribution does not include JVM-core. You need to add it to your cluster or Spark-Connect server!* +**NOTE!** *GraphFrames depends on its own version of GraphX. While in case of installation from the Maven Central repository, all the runtime dependencies will be resolved automatically, you may need to add them manually in case of installation from the local repository.* + ## Running `graphframes-py` You should use GraphFrames via the `--packages` argument to `pyspark` or `spark-submit`, but this package is helpful in development environments. diff --git a/python/tests/conftest.py b/python/tests/conftest.py index 1fee7c475..c4c87ca1b 100644 --- a/python/tests/conftest.py +++ b/python/tests/conftest.py @@ -22,7 +22,7 @@ def is_remote() -> bool: scala_version = os.environ.get("SCALA_VERSION", "2.12" if __version__ < "4" else "2.13") -def get_gf_jar_locations() -> Tuple[str, str]: +def get_gf_jar_locations() -> Tuple[str, str, str]: """ Returns a location of the GraphFrames JAR and GraphFrames Connect JAR. @@ -30,12 +30,23 @@ def get_gf_jar_locations() -> Tuple[str, str]: this function will raise an exception! """ project_root = pathlib.Path(__file__).parent.parent.parent + graphx_dir = project_root / "graphx" / "target" / f"scala-{scala_version}" core_dir = project_root / "core" / "target" / f"scala-{scala_version}" connect_dir = project_root / "connect" / "target" / f"scala-{scala_version}" + graphx_jar: Optional[str] = None core_jar: Optional[str] = None connect_jar: Optional[str] = None + for pp in graphx_dir.glob(f"graphframes-graphx-spark{spark_major_version}*.jar"): + assert isinstance(pp, pathlib.PosixPath) # type checking + graphx_jar = str(pp.absolute()) + + if graphx_jar is None: + raise ValueError( + f"Failed to find graphframes jar for Spark {spark_major_version} in {graphx_dir}" + ) + for pp in core_dir.glob(f"graphframes-spark{spark_major_version}*.jar"): assert isinstance(pp, pathlib.PosixPath) # type checking core_jar = str(pp.absolute()) @@ -54,7 +65,7 @@ def get_gf_jar_locations() -> Tuple[str, str]: f"Failed to find graphframes connect jar for Spark {spark_major_version} in {connect_dir}" ) - return (core_jar, connect_jar) + return core_jar, connect_jar, graphx_jar @pytest.fixture(scope="module") @@ -62,14 +73,14 @@ def spark(): warnings.filterwarnings("ignore", category=ResourceWarning) warnings.filterwarnings("ignore", category=DeprecationWarning) - (core_jar, connect_jar) = get_gf_jar_locations() + (core_jar, connect_jar, graphx_jar) = get_gf_jar_locations() with tempfile.TemporaryDirectory() as tmp_dir: builder = (SparkSession.Builder() .appName("GraphFramesTest") .config("spark.sql.shuffle.partitions", 4) .config("spark.checkpoint.dir", tmp_dir) - .config("spark.jars", f"{core_jar},{connect_jar}") + .config("spark.jars", f"{core_jar},{connect_jar},{graphx_jar}") ) if spark_major_version == "3":