public class GraphImpl<VD,ED> extends Graph<VD,ED> implements scala.Serializable
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.
Modifier and Type | Method and Description |
---|---|
<A> VertexRDD<A> |
aggregateMessagesWithActiveSet(scala.Function1<EdgeContext<VD,ED,A>,scala.runtime.BoxedUnit> sendMsg,
scala.Function2<A,A,A> mergeMsg,
TripletFields tripletFields,
scala.Option<scala.Tuple2<VertexRDD<?>,EdgeDirection>> activeSetOpt,
scala.reflect.ClassTag<A> evidence$11) |
static <VD,ED> GraphImpl<VD,ED> |
apply(RDD<Edge<ED>> edges,
VD defaultVertexAttr,
StorageLevel edgeStorageLevel,
StorageLevel vertexStorageLevel,
scala.reflect.ClassTag<VD> evidence$14,
scala.reflect.ClassTag<ED> evidence$15)
Create a graph from edges, setting referenced vertices to `defaultVertexAttr`.
|
static <VD,ED> GraphImpl<VD,ED> |
apply(RDD<scala.Tuple2<Object,VD>> vertices,
RDD<Edge<ED>> edges,
VD defaultVertexAttr,
StorageLevel edgeStorageLevel,
StorageLevel vertexStorageLevel,
scala.reflect.ClassTag<VD> evidence$18,
scala.reflect.ClassTag<ED> evidence$19)
Create a graph from vertices and edges, setting missing vertices to `defaultVertexAttr`.
|
static <VD,ED> GraphImpl<VD,ED> |
apply(VertexRDD<VD> vertices,
EdgeRDD<ED> edges,
scala.reflect.ClassTag<VD> evidence$20,
scala.reflect.ClassTag<ED> evidence$21)
Create a graph from a VertexRDD and an EdgeRDD with arbitrary replicated vertices.
|
Graph<VD,ED> |
cache()
Caches the vertices and edges associated with this graph at the previously-specified target
storage levels, which default to
MEMORY_ONLY . |
void |
checkpoint()
Mark this Graph for checkpointing.
|
EdgeRDDImpl<ED,VD> |
edges()
An RDD containing the edges and their associated attributes.
|
static <VD,ED> GraphImpl<VD,ED> |
fromEdgePartitions(RDD<scala.Tuple2<Object,org.apache.spark.graphx.impl.EdgePartition<ED,VD>>> edgePartitions,
VD defaultVertexAttr,
StorageLevel edgeStorageLevel,
StorageLevel vertexStorageLevel,
scala.reflect.ClassTag<VD> evidence$16,
scala.reflect.ClassTag<ED> evidence$17)
Create a graph from EdgePartitions, setting referenced vertices to `defaultVertexAttr`.
|
static <VD,ED> GraphImpl<VD,ED> |
fromExistingRDDs(VertexRDD<VD> vertices,
EdgeRDD<ED> edges,
scala.reflect.ClassTag<VD> evidence$22,
scala.reflect.ClassTag<ED> evidence$23)
Create a graph from a VertexRDD and an EdgeRDD with the same replicated vertex type as the
vertices.
|
scala.collection.Seq<String> |
getCheckpointFiles()
Gets the name of the files to which this Graph was checkpointed.
|
Graph<VD,ED> |
groupEdges(scala.Function2<ED,ED,ED> merge)
Merges multiple edges between two vertices into a single edge.
|
boolean |
isCheckpointed()
Return whether this Graph has been checkpointed or not.
|
<ED2> Graph<VD,ED2> |
mapEdges(scala.Function2<Object,scala.collection.Iterator<Edge<ED>>,scala.collection.Iterator<ED2>> f,
scala.reflect.ClassTag<ED2> evidence$6)
Transforms each edge attribute using the map function, passing it a whole partition at a
time.
|
<A> VertexRDD<A> |
mapReduceTriplets(scala.Function1<EdgeTriplet<VD,ED>,scala.collection.Iterator<scala.Tuple2<Object,A>>> mapFunc,
scala.Function2<A,A,A> reduceFunc,
scala.Option<scala.Tuple2<VertexRDD<?>,EdgeDirection>> activeSetOpt,
scala.reflect.ClassTag<A> evidence$10)
Aggregates values from the neighboring edges and vertices of each vertex.
|
<ED2> Graph<VD,ED2> |
mapTriplets(scala.Function2<Object,scala.collection.Iterator<EdgeTriplet<VD,ED>>,scala.collection.Iterator<ED2>> f,
TripletFields tripletFields,
scala.reflect.ClassTag<ED2> evidence$7)
Transforms each edge attribute a partition at a time using the map function, passing it the
adjacent vertex attributes as well.
|
<VD2> Graph<VD2,ED> |
mapVertices(scala.Function2<Object,VD,VD2> f,
scala.reflect.ClassTag<VD2> evidence$5,
scala.Predef.$eq$colon$eq<VD,VD2> eq)
Transforms each vertex attribute in the graph using the map function.
|
<VD2,ED2> Graph<VD,ED> |
mask(Graph<VD2,ED2> other,
scala.reflect.ClassTag<VD2> evidence$8,
scala.reflect.ClassTag<ED2> evidence$9)
Restricts the graph to only the vertices and edges that are also in
other , but keeps the
attributes from this graph. |
<U,VD2> Graph<VD2,ED> |
outerJoinVertices(RDD<scala.Tuple2<Object,U>> other,
scala.Function3<Object,VD,scala.Option<U>,VD2> updateF,
scala.reflect.ClassTag<U> evidence$12,
scala.reflect.ClassTag<VD2> evidence$13,
scala.Predef.$eq$colon$eq<VD,VD2> eq)
Joins the vertices with entries in the
table RDD and merges the results using mapFunc . |
Graph<VD,ED> |
partitionBy(PartitionStrategy partitionStrategy)
Repartitions the edges in the graph according to
partitionStrategy . |
Graph<VD,ED> |
partitionBy(PartitionStrategy partitionStrategy,
int numPartitions)
Repartitions the edges in the graph according to
partitionStrategy . |
Graph<VD,ED> |
persist(StorageLevel newLevel)
Caches the vertices and edges associated with this graph at the specified storage level,
ignoring any target storage levels previously set.
|
org.apache.spark.graphx.impl.ReplicatedVertexView<VD,ED> |
replicatedVertexView() |
Graph<VD,ED> |
reverse()
Reverses all edges in the graph.
|
Graph<VD,ED> |
subgraph(scala.Function1<EdgeTriplet<VD,ED>,Object> epred,
scala.Function2<Object,VD,Object> vpred)
Restricts the graph to only the vertices and edges satisfying the predicates.
|
RDD<EdgeTriplet<VD,ED>> |
triplets()
Return a RDD that brings edges together with their source and destination vertices.
|
Graph<VD,ED> |
unpersist(boolean blocking)
Uncaches both vertices and edges of this graph.
|
Graph<VD,ED> |
unpersistVertices(boolean blocking)
Uncaches only the vertices of this graph, leaving the edges alone.
|
VertexRDD<VD> |
vertices()
An RDD containing the vertices and their associated attributes.
|
aggregateMessages, fromEdges, fromEdgeTuples, graphToGraphOps, mapEdges, mapTriplets, mapTriplets, ops
public static <VD,ED> GraphImpl<VD,ED> apply(RDD<Edge<ED>> edges, VD defaultVertexAttr, StorageLevel edgeStorageLevel, StorageLevel vertexStorageLevel, scala.reflect.ClassTag<VD> evidence$14, scala.reflect.ClassTag<ED> evidence$15)
public static <VD,ED> GraphImpl<VD,ED> fromEdgePartitions(RDD<scala.Tuple2<Object,org.apache.spark.graphx.impl.EdgePartition<ED,VD>>> edgePartitions, VD defaultVertexAttr, StorageLevel edgeStorageLevel, StorageLevel vertexStorageLevel, scala.reflect.ClassTag<VD> evidence$16, scala.reflect.ClassTag<ED> evidence$17)
public static <VD,ED> GraphImpl<VD,ED> apply(RDD<scala.Tuple2<Object,VD>> vertices, RDD<Edge<ED>> edges, VD defaultVertexAttr, StorageLevel edgeStorageLevel, StorageLevel vertexStorageLevel, scala.reflect.ClassTag<VD> evidence$18, scala.reflect.ClassTag<ED> evidence$19)
public static <VD,ED> GraphImpl<VD,ED> apply(VertexRDD<VD> vertices, EdgeRDD<ED> edges, scala.reflect.ClassTag<VD> evidence$20, scala.reflect.ClassTag<ED> evidence$21)
VertexRDD.withEdges
or an appropriate VertexRDD constructor.vertices
- (undocumented)edges
- (undocumented)evidence$20
- (undocumented)evidence$21
- (undocumented)public static <VD,ED> GraphImpl<VD,ED> fromExistingRDDs(VertexRDD<VD> vertices, EdgeRDD<ED> edges, scala.reflect.ClassTag<VD> evidence$22, scala.reflect.ClassTag<ED> evidence$23)
VertexRDD.withEdges
or an appropriate VertexRDD constructor.vertices
- (undocumented)edges
- (undocumented)evidence$22
- (undocumented)evidence$23
- (undocumented)public VertexRDD<VD> vertices()
Graph
public org.apache.spark.graphx.impl.ReplicatedVertexView<VD,ED> replicatedVertexView()
public EdgeRDDImpl<ED,VD> edges()
Graph
public RDD<EdgeTriplet<VD,ED>> triplets()
public Graph<VD,ED> persist(StorageLevel newLevel)
Graph
public Graph<VD,ED> cache()
Graph
MEMORY_ONLY
. This is used to pin a graph in memory enabling
multiple queries to reuse the same construction process.public void checkpoint()
Graph
checkpoint
in class Graph<VD,ED>
public boolean isCheckpointed()
Graph
isCheckpointed
in class Graph<VD,ED>
public scala.collection.Seq<String> getCheckpointFiles()
Graph
getCheckpointFiles
in class Graph<VD,ED>
public Graph<VD,ED> unpersist(boolean blocking)
Graph
public Graph<VD,ED> unpersistVertices(boolean blocking)
Graph
unpersistVertices
in class Graph<VD,ED>
blocking
- (undocumented)public Graph<VD,ED> partitionBy(PartitionStrategy partitionStrategy)
Graph
partitionStrategy
.
partitionBy
in class Graph<VD,ED>
partitionStrategy
- the partitioning strategy to use when partitioning the edges
in the graph.public Graph<VD,ED> partitionBy(PartitionStrategy partitionStrategy, int numPartitions)
Graph
partitionStrategy
.
partitionBy
in class Graph<VD,ED>
partitionStrategy
- the partitioning strategy to use when partitioning the edges
in the graph.numPartitions
- the number of edge partitions in the new graph.public Graph<VD,ED> reverse()
Graph
public <VD2> Graph<VD2,ED> mapVertices(scala.Function2<Object,VD,VD2> f, scala.reflect.ClassTag<VD2> evidence$5, scala.Predef.$eq$colon$eq<VD,VD2> eq)
Graph
mapVertices
in class Graph<VD,ED>
f
- the function from a vertex object to a new vertex value
evidence$5
- (undocumented)eq
- (undocumented)public <ED2> Graph<VD,ED2> mapEdges(scala.Function2<Object,scala.collection.Iterator<Edge<ED>>,scala.collection.Iterator<ED2>> f, scala.reflect.ClassTag<ED2> evidence$6)
Graph
mapTriplets
.
public <ED2> Graph<VD,ED2> mapTriplets(scala.Function2<Object,scala.collection.Iterator<EdgeTriplet<VD,ED>>,scala.collection.Iterator<ED2>> f, TripletFields tripletFields, scala.reflect.ClassTag<ED2> evidence$7)
Graph
mapEdges
instead.
mapTriplets
in class Graph<VD,ED>
f
- the iterator transformtripletFields
- 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.
evidence$7
- (undocumented)public Graph<VD,ED> subgraph(scala.Function1<EdgeTriplet<VD,ED>,Object> epred, scala.Function2<Object,VD,Object> vpred)
Graph
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)}
subgraph
in class Graph<VD,ED>
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
public <VD2,ED2> Graph<VD,ED> mask(Graph<VD2,ED2> other, scala.reflect.ClassTag<VD2> evidence$8, scala.reflect.ClassTag<ED2> evidence$9)
Graph
other
, but keeps the
attributes from this graph.public Graph<VD,ED> groupEdges(scala.Function2<ED,ED,ED> merge)
Graph
partitionBy
.
groupEdges
in class Graph<VD,ED>
merge
- the user-supplied commutative associative function to merge edge attributes
for duplicate edges.
public <A> VertexRDD<A> mapReduceTriplets(scala.Function1<EdgeTriplet<VD,ED>,scala.collection.Iterator<scala.Tuple2<Object,A>>> mapFunc, scala.Function2<A,A,A> reduceFunc, scala.Option<scala.Tuple2<VertexRDD<?>,EdgeDirection>> activeSetOpt, scala.reflect.ClassTag<A> evidence$10)
Graph
mapFunc
function is invoked on each edge of the graph, generating 0 or more "messages" to be
"sent" to either vertex in the edge. The reduceFunc
is then used to combine the output of
the map phase destined to each vertex.
This function is deprecated in 1.2.0 because of SPARK-3936. Use aggregateMessages instead.
mapReduceTriplets
in class Graph<VD,ED>
mapFunc
- the user defined map function which returns 0 or
more messages to neighboring vertices
reduceFunc
- the user defined reduce function which should
be commutative and associative and is used to combine the output
of the map phase
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 only on 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.
evidence$10
- (undocumented)public <A> VertexRDD<A> aggregateMessagesWithActiveSet(scala.Function1<EdgeContext<VD,ED,A>,scala.runtime.BoxedUnit> sendMsg, scala.Function2<A,A,A> mergeMsg, TripletFields tripletFields, scala.Option<scala.Tuple2<VertexRDD<?>,EdgeDirection>> activeSetOpt, scala.reflect.ClassTag<A> evidence$11)
public <U,VD2> Graph<VD2,ED> outerJoinVertices(RDD<scala.Tuple2<Object,U>> other, scala.Function3<Object,VD,scala.Option<U>,VD2> updateF, scala.reflect.ClassTag<U> evidence$12, scala.reflect.ClassTag<VD2> evidence$13, scala.Predef.$eq$colon$eq<VD,VD2> eq)
Graph
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
.
outerJoinVertices
in class Graph<VD,ED>
other
- the table to join with the vertices in the graph.
The table should contain at most one entry for each vertex.updateF
- 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.
evidence$12
- (undocumented)evidence$13
- (undocumented)eq
- (undocumented)