abstract class Graph[VD, ED] extends Serializable
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.
- VD
- the vertex attribute type 
- ED
- the edge attribute type 
- Source
- Graph.scala
- Note
- GraphOps contains additional convenience operations and graph algorithms. 
- Alphabetic
- By Inheritance
- Graph
- Serializable
- Serializable
- AnyRef
- Any
- Hide All
- Show All
- Public
- All
Instance Constructors
- 
      
      
      
        
      
    
      
        
        new
      
      
        Graph()(implicit arg0: ClassTag[VD], arg1: ClassTag[ED])
      
      
      - Attributes
- protected
 
Abstract Value Members
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        cache(): Graph[VD, ED]
      
      
      Caches the vertices and edges associated with this graph at the previously-specified target storage levels, which default to MEMORY_ONLY.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.
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        checkpoint(): Unit
      
      
      Mark this Graph for checkpointing. 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. 
- 
      
      
      
        
      
    
      
        abstract 
        val
      
      
        edges: EdgeRDD[ED]
      
      
      An RDD containing the edges and their associated attributes. 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. - returns
- an RDD containing the edges in this graph 
 - See also
- Edgefor the edge type.- Graph#tripletsto get an RDD which contains all the edges along with their vertex data.
 
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        getCheckpointFiles: Seq[String]
      
      
      Gets the name of the files to which this Graph was checkpointed. Gets the name of the files to which this Graph was checkpointed. (The vertices RDD and edges RDD are checkpointed separately.) 
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        groupEdges(merge: (ED, ED) ⇒ ED): Graph[VD, ED]
      
      
      Merges multiple edges between two vertices into a single edge. Merges multiple edges between two vertices into a single edge. For correct results, the graph must have been partitioned using partitionBy.- merge
- the user-supplied commutative associative function to merge edge attributes for duplicate edges. 
- returns
- The resulting graph with a single edge for each (source, dest) vertex pair. 
 
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        isCheckpointed: Boolean
      
      
      Return whether this Graph has been checkpointed or not. Return whether this Graph has been checkpointed or not. This returns true iff both the vertices RDD and edges RDD have been checkpointed. 
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        mapEdges[ED2](map: (PartitionID, Iterator[Edge[ED]]) ⇒ Iterator[ED2])(implicit arg0: ClassTag[ED2]): Graph[VD, ED2]
      
      
      Transforms each edge attribute using the map function, passing it a whole partition at a time. 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.- ED2
- the new edge data type 
- 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 
 - 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. 
 
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        mapTriplets[ED2](map: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) ⇒ Iterator[ED2], tripletFields: TripletFields)(implicit arg0: ClassTag[ED2]): Graph[VD, ED2]
      
      
      Transforms each edge attribute a partition at a time using the map function, passing it the adjacent vertex attributes as well. 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 mapEdgesinstead.- ED2
- the new edge data type 
- map
- the iterator transform 
- 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. 
 - 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. 
 
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        mapVertices[VD2](map: (VertexId, VD) ⇒ VD2)(implicit arg0: ClassTag[VD2], eq: =:=[VD, VD2] = null): Graph[VD2, ED]
      
      
      Transforms each vertex attribute in the graph using the map function. Transforms each vertex attribute in the graph using the map function. - VD2
- the new vertex data type 
- map
- the function from a vertex object to a new vertex value 
 - 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) 
- Note
- The new graph has the same structure. As a consequence the underlying index structures can be reused. 
 Example:
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        mask[VD2, ED2](other: Graph[VD2, ED2])(implicit arg0: ClassTag[VD2], arg1: ClassTag[ED2]): Graph[VD, ED]
      
      
      Restricts the graph to only the vertices and edges that are also in other, but keeps the attributes from this graph.Restricts the graph to only the vertices and edges that are also in other, but keeps the attributes from this graph.- other
- the graph to project this graph onto 
- returns
- a graph with vertices and edges that exist in both the current graph and - other, with vertex and edge data from the current graph
 
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        outerJoinVertices[U, VD2](other: RDD[(VertexId, U)])(mapFunc: (VertexId, VD, Option[U]) ⇒ VD2)(implicit arg0: ClassTag[U], arg1: ClassTag[VD2], eq: =:=[VD, VD2] = null): Graph[VD2, ED]
      
      
      Joins the vertices with entries in the tableRDD and merges the results usingmapFunc.Joins the vertices with entries in the tableRDD and merges the results usingmapFunc. The input table should contain at most one entry for each vertex. If no entry inotheris provided for a particular vertex in the graph, the map function receivesNone.- U
- the type of entry in the table of updates 
- VD2
- the new vertex value type 
- other
- the table to join with the vertices in the graph. The table should contain at most one entry for each vertex. 
- 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. 
 - 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) } 
 Example:
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        partitionBy(partitionStrategy: PartitionStrategy, numPartitions: Int): Graph[VD, ED]
      
      
      Repartitions the edges in the graph according to partitionStrategy.Repartitions the edges in the graph according to partitionStrategy.- partitionStrategy
- the partitioning strategy to use when partitioning the edges in the graph. 
- numPartitions
- the number of edge partitions in the new graph. 
 
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED]
      
      
      Repartitions the edges in the graph according to partitionStrategy.Repartitions the edges in the graph according to partitionStrategy.- partitionStrategy
- the partitioning strategy to use when partitioning the edges in the graph. 
 
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        persist(newLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED]
      
      
      Caches the vertices and edges associated with this graph at the specified storage level, ignoring any target storage levels previously set. Caches the vertices and edges associated with this graph at the specified storage level, ignoring any target storage levels previously set. - newLevel
- the level at which to cache the graph. 
- returns
- A reference to this graph for convenience. 
 
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        reverse: Graph[VD, ED]
      
      
      Reverses all edges in the graph. 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. 
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        subgraph(epred: (EdgeTriplet[VD, ED]) ⇒ Boolean = x => true, vpred: (VertexId, VD) ⇒ Boolean = (v, d) => true): Graph[VD, ED]
      
      
      Restricts the graph to only the vertices and edges satisfying the predicates. 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)}- 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. 
- vpred
- the vertex predicate, which takes a vertex object and evaluates to true if the vertex is to be included in the subgraph 
- returns
- the subgraph containing only the vertices and edges that satisfy the predicates 
 
- 
      
      
      
        
      
    
      
        abstract 
        val
      
      
        triplets: RDD[EdgeTriplet[VD, ED]]
      
      
      An RDD containing the edge triplets, which are edges along with the vertex data associated with the adjacent vertices. 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. - returns
- an RDD containing edge triplets 
 - 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 
 Example:
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        unpersist(blocking: Boolean = false): Graph[VD, ED]
      
      
      Uncaches both vertices and edges of this graph. Uncaches both vertices and edges of this graph. This is useful in iterative algorithms that build a new graph in each iteration. - blocking
- Whether to block until all data is unpersisted (default: false) 
 
- 
      
      
      
        
      
    
      
        abstract 
        def
      
      
        unpersistVertices(blocking: Boolean = false): Graph[VD, ED]
      
      
      Uncaches only the vertices of this graph, leaving the edges alone. 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. - blocking
- Whether to block until all data is unpersisted (default: false) 
 
- 
      
      
      
        
      
    
      
        abstract 
        val
      
      
        vertices: VertexRDD[VD]
      
      
      An RDD containing the vertices and their associated attributes. An RDD containing the vertices and their associated attributes. - returns
- an RDD containing the vertices in this graph 
 - Note
- vertex ids are unique. 
 
Concrete Value Members
- 
      
      
      
        
      
    
      
        final 
        def
      
      
        !=(arg0: Any): Boolean
      
      
      - Definition Classes
- AnyRef → Any
 
- 
      
      
      
        
      
    
      
        final 
        def
      
      
        ##(): Int
      
      
      - Definition Classes
- AnyRef → Any
 
- 
      
      
      
        
      
    
      
        final 
        def
      
      
        ==(arg0: Any): Boolean
      
      
      - Definition Classes
- AnyRef → Any
 
- 
      
      
      
        
      
    
      
        
        def
      
      
        aggregateMessages[A](sendMsg: (EdgeContext[VD, ED, A]) ⇒ Unit, mergeMsg: (A, A) ⇒ A, tripletFields: TripletFields = TripletFields.All)(implicit arg0: ClassTag[A]): VertexRDD[A]
      
      
      Aggregates values from the neighboring edges and vertices of each vertex. Aggregates values from the neighboring edges and vertices of each vertex. The user-supplied sendMsgfunction is invoked on each edge of the graph, generating 0 or more messages to be sent to either vertex in the edge. ThemergeMsgfunction is then used to combine all messages destined to the same vertex.- A
- the type of message to be sent to each vertex 
- sendMsg
- runs on each edge, sending messages to neighboring vertices using the EdgeContext. 
- mergeMsg
- used to combine messages from - sendMsgdestined to the same vertex. This combiner should be commutative and associative.
- tripletFields
- which fields should be included in the EdgeContext passed to the - sendMsgfunction. If not all fields are needed, specifying this can improve performance.
 - 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. 
 Example:
- 
      
      
      
        
      
    
      
        final 
        def
      
      
        asInstanceOf[T0]: T0
      
      
      - Definition Classes
- Any
 
- 
      
      
      
        
      
    
      
        
        def
      
      
        clone(): AnyRef
      
      
      - Attributes
- protected[lang]
- Definition Classes
- AnyRef
- Annotations
- @throws( ... ) @native() @IntrinsicCandidate()
 
- 
      
      
      
        
      
    
      
        final 
        def
      
      
        eq(arg0: AnyRef): Boolean
      
      
      - Definition Classes
- AnyRef
 
- 
      
      
      
        
      
    
      
        
        def
      
      
        equals(arg0: Any): Boolean
      
      
      - Definition Classes
- AnyRef → Any
 
- 
      
      
      
        
      
    
      
        final 
        def
      
      
        getClass(): Class[_]
      
      
      - Definition Classes
- AnyRef → Any
- Annotations
- @native() @IntrinsicCandidate()
 
- 
      
      
      
        
      
    
      
        
        def
      
      
        hashCode(): Int
      
      
      - Definition Classes
- AnyRef → Any
- Annotations
- @native() @IntrinsicCandidate()
 
- 
      
      
      
        
      
    
      
        final 
        def
      
      
        isInstanceOf[T0]: Boolean
      
      
      - Definition Classes
- Any
 
- 
      
      
      
        
      
    
      
        
        def
      
      
        mapEdges[ED2](map: (Edge[ED]) ⇒ ED2)(implicit arg0: ClassTag[ED2]): Graph[VD, ED2]
      
      
      Transforms each edge attribute in the graph using the map function. 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.- ED2
- the new edge data type 
- map
- the function from an edge object to a new edge value. 
 - This function might be used to initialize edge attributes. 
- 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. 
 Example:
- 
      
      
      
        
      
    
      
        
        def
      
      
        mapTriplets[ED2](map: (EdgeTriplet[VD, ED]) ⇒ ED2, tripletFields: TripletFields)(implicit arg0: ClassTag[ED2]): Graph[VD, ED2]
      
      
      Transforms each edge attribute using the map function, passing it the adjacent vertex attributes as well. 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 mapEdgesinstead.- ED2
- the new edge data type 
- map
- the function from an edge object to a new edge value. 
- 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. 
 - 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) 
- 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. 
 Example:
- 
      
      
      
        
      
    
      
        
        def
      
      
        mapTriplets[ED2](map: (EdgeTriplet[VD, ED]) ⇒ ED2)(implicit arg0: ClassTag[ED2]): Graph[VD, ED2]
      
      
      Transforms each edge attribute using the map function, passing it the adjacent vertex attributes as well. 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 mapEdgesinstead.- ED2
- the new edge data type 
- map
- the function from an edge object to a new edge value. 
 - 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) 
- 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. 
 Example:
- 
      
      
      
        
      
    
      
        final 
        def
      
      
        ne(arg0: AnyRef): Boolean
      
      
      - Definition Classes
- AnyRef
 
- 
      
      
      
        
      
    
      
        final 
        def
      
      
        notify(): Unit
      
      
      - Definition Classes
- AnyRef
- Annotations
- @native() @IntrinsicCandidate()
 
- 
      
      
      
        
      
    
      
        final 
        def
      
      
        notifyAll(): Unit
      
      
      - Definition Classes
- AnyRef
- Annotations
- @native() @IntrinsicCandidate()
 
- 
      
      
      
        
      
    
      
        
        val
      
      
        ops: GraphOps[VD, ED]
      
      
      The associated GraphOps object. 
- 
      
      
      
        
      
    
      
        final 
        def
      
      
        synchronized[T0](arg0: ⇒ T0): T0
      
      
      - Definition Classes
- AnyRef
 
- 
      
      
      
        
      
    
      
        
        def
      
      
        toString(): String
      
      
      - Definition Classes
- AnyRef → Any
 
- 
      
      
      
        
      
    
      
        final 
        def
      
      
        wait(arg0: Long, arg1: Int): Unit
      
      
      - Definition Classes
- AnyRef
- Annotations
- @throws( ... )
 
- 
      
      
      
        
      
    
      
        final 
        def
      
      
        wait(arg0: Long): Unit
      
      
      - Definition Classes
- AnyRef
- Annotations
- @throws( ... ) @native()
 
- 
      
      
      
        
      
    
      
        final 
        def
      
      
        wait(): Unit
      
      
      - Definition Classes
- AnyRef
- Annotations
- @throws( ... )
 
Deprecated Value Members
- 
      
      
      
        
      
    
      
        
        def
      
      
        finalize(): Unit
      
      
      - Attributes
- protected[lang]
- Definition Classes
- AnyRef
- Annotations
- @throws( classOf[java.lang.Throwable] ) @Deprecated
- Deprecated