-
Notifications
You must be signed in to change notification settings - Fork 266
Expand file tree
/
Copy pathgraphframe.py
More file actions
448 lines (359 loc) · 15.9 KB
/
graphframe.py
File metadata and controls
448 lines (359 loc) · 15.9 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
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.
#
from __future__ import annotations
from typing import TYPE_CHECKING, Any, Optional
from pyspark.storagelevel import StorageLevel
from pyspark.version import __version__
if __version__[:3] >= "3.4":
from pyspark.sql.utils import is_remote
else:
# All the Connect-related utilities are accessible starting from 3.4.x
def is_remote() -> bool:
return False
from pyspark.sql import SparkSession
from graphframes.classic.graphframe import GraphFrame as GraphFrameClassic
from graphframes.lib import Pregel
if TYPE_CHECKING:
from pyspark.sql import Column, DataFrame
from graphframes.connect.graphframe_client import GraphFrameConnect
class GraphFrame:
"""
Represents a graph with vertices and edges stored as DataFrames.
:param v: :class:`DataFrame` holding vertex information.
Must contain a column named "id" that stores unique
vertex IDs.
:param e: :class:`DataFrame` holding edge information.
Must contain two columns "src" and "dst" storing source
vertex IDs and destination vertex IDs of edges, respectively.
>>> localVertices = [(1,"A"), (2,"B"), (3, "C")]
>>> localEdges = [(1,2,"love"), (2,1,"hate"), (2,3,"follow")]
>>> v = spark.createDataFrame(localVertices, ["id", "name"])
>>> e = spark.createDataFrame(localEdges, ["src", "dst", "action"])
>>> g = GraphFrame(v, e)
"""
@staticmethod
def _from_impl(impl: GraphFrameClassic | "GraphFrameConnect") -> "GraphFrame":
return GraphFrame(impl.vertices, impl.edges)
def __init__(self, v: DataFrame, e: DataFrame) -> None:
self._impl: GraphFrameClassic | "GraphFrameConnect"
if is_remote():
from graphframes.connect.graphframe_client import GraphFrameConnect
self._impl = GraphFrameConnect(v, e)
else:
self._impl = GraphFrameClassic(v, e)
@property
def vertices(self) -> DataFrame:
"""
:class:`DataFrame` holding vertex information, with unique column "id"
for vertex IDs.
"""
return self._impl.vertices
@property
def edges(self) -> DataFrame:
"""
:class:`DataFrame` holding edge information, with unique columns "src" and
"dst" storing source vertex IDs and destination vertex IDs of edges,
respectively.
"""
return self._impl.edges
def __repr__(self) -> str:
return self._impl.__repr__()
def cache(self) -> "GraphFrame":
"""Persist the dataframe representation of vertices and edges of the graph with the default
storage level.
"""
return GraphFrame._from_impl(self._impl.cache())
def persist(self, storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) -> "GraphFrame":
"""Persist the dataframe representation of vertices and edges of the graph with the given
storage level.
"""
return GraphFrame._from_impl(self._impl.persist(storageLevel=storageLevel))
def unpersist(self, blocking: bool = False) -> "GraphFrame":
"""Mark the dataframe representation of vertices and edges of the graph as non-persistent,
and remove all blocks for it from memory and disk.
"""
return GraphFrame._from_impl(self._impl.unpersist(blocking=blocking))
@property
def outDegrees(self) -> DataFrame:
"""
The out-degree of each vertex in the graph, returned as a DataFrame with two columns:
- "id": the ID of the vertex
- "outDegree" (integer) storing the out-degree of the vertex
Note that vertices with 0 out-edges are not returned in the result.
:return: DataFrame with new vertices column "outDegree"
"""
return self._impl.outDegrees
@property
def inDegrees(self) -> DataFrame:
"""
The in-degree of each vertex in the graph, returned as a DataFame with two columns:
- "id": the ID of the vertex
- "inDegree" (int) storing the in-degree of the vertex
Note that vertices with 0 in-edges are not returned in the result.
:return: DataFrame with new vertices column "inDegree"
"""
return self._impl.inDegrees
@property
def degrees(self) -> DataFrame:
"""
The degree of each vertex in the graph, returned as a DataFrame with two columns:
- "id": the ID of the vertex
- 'degree' (integer) the degree of the vertex
Note that vertices with 0 edges are not returned in the result.
:return: DataFrame with new vertices column "degree"
"""
return self._impl.degrees
@property
def triplets(self) -> DataFrame:
"""
The triplets (source vertex)-[edge]->(destination vertex) for all edges in the graph.
Returned as a :class:`DataFrame` with three columns:
- "src": source vertex with schema matching 'vertices'
- "edge": edge with schema matching 'edges'
- 'dst': destination vertex with schema matching 'vertices'
:return: DataFrame with columns 'src', 'edge', and 'dst'
"""
return self._impl.triplets
@property
def pregel(self) -> Pregel:
"""
Get the :class:`graphframes.lib.Pregel` object for running pregel.
See :class:`graphframes.lib.Pregel` for more details.
"""
return self._impl.pregel
def find(self, pattern: str) -> DataFrame:
"""
Motif finding.
See Scala documentation for more details.
:param pattern: String describing the motif to search for.
:return: DataFrame with one Row for each instance of the motif found
"""
return self._impl.find(pattern=pattern)
def filterVertices(self, condition: str | Column) -> "GraphFrame":
"""
Filters the vertices based on expression, remove edges containing any dropped vertices.
:param condition: String or Column describing the condition expression for filtering.
:return: GraphFrame with filtered vertices and edges.
"""
return GraphFrame._from_impl(self._impl.filterVertices(condition=condition))
def filterEdges(self, condition: str | Column) -> "GraphFrame":
"""
Filters the edges based on expression, keep all vertices.
:param condition: String or Column describing the condition expression for filtering.
:return: GraphFrame with filtered edges.
"""
return GraphFrame._from_impl(self._impl.filterEdges(condition=condition))
def dropIsolatedVertices(self) -> "GraphFrame":
"""
Drops isolated vertices, vertices are not contained in any edges.
:return: GraphFrame with filtered vertices.
"""
return GraphFrame._from_impl(self._impl.dropIsolatedVertices())
def bfs(
self,
fromExpr: str,
toExpr: str,
edgeFilter: str | None = None,
maxPathLength: int = 10,
) -> DataFrame:
"""
Breadth-first search (BFS).
See Scala documentation for more details.
:return: DataFrame with one Row for each shortest path between matching vertices.
"""
return self._impl.bfs(
fromExpr=fromExpr,
toExpr=toExpr,
edgeFilter=edgeFilter,
maxPathLength=maxPathLength,
)
def aggregateMessages(
self,
aggCol: Column | str,
sendToSrc: Column | str | None = None,
sendToDst: Column | str | None = None,
) -> DataFrame:
"""
Aggregates messages from the neighbours.
When specifying the messages and aggregation function, the user may reference columns using
the static methods in :class:`graphframes.lib.AggregateMessages`.
See Scala documentation for more details.
:param aggCol: the requested aggregation output either as
:class:`pyspark.sql.Column` or SQL expression string
:param sendToSrc: message sent to the source vertex of each triplet either as
:class:`pyspark.sql.Column` or SQL expression string (default: None)
:param sendToDst: message sent to the destination vertex of each triplet either as
:class:`pyspark.sql.Column` or SQL expression string (default: None)
:return: DataFrame with columns for the vertex ID and the resulting aggregated message
"""
return self._impl.aggregateMessages(aggCol=aggCol, sendToSrc=sendToSrc, sendToDst=sendToDst)
# Standard algorithms
def connectedComponents(
self,
algorithm: str = "graphframes",
checkpointInterval: int = 2,
broadcastThreshold: int = 1000000,
useLabelsAsComponents: bool = False,
) -> DataFrame:
"""
Computes the connected components of the graph.
See Scala documentation for more details.
:param algorithm: connected components algorithm to use (default: "graphframes")
Supported algorithms are "graphframes" and "graphx".
:param checkpointInterval: checkpoint interval in terms of number of iterations (default: 2)
:param broadcastThreshold: broadcast threshold in propagating component assignments
(default: 1000000)
:param useLabelsAsComponents: if True, uses the vertex labels as components, otherwise will
use longs
:return: DataFrame with new vertices column "component"
"""
return self._impl.connectedComponents(
algorithm=algorithm,
checkpointInterval=checkpointInterval,
broadcastThreshold=broadcastThreshold,
useLabelsAsComponents=useLabelsAsComponents,
)
def labelPropagation(self, maxIter: int) -> DataFrame:
"""
Runs static label propagation for detecting communities in networks.
See Scala documentation for more details.
:param maxIter: the number of iterations to be performed
:return: DataFrame with new vertices column "label"
"""
return self._impl.labelPropagation(maxIter=maxIter)
def pageRank(
self,
resetProbability: float = 0.15,
sourceId: Optional[Any] = None,
maxIter: Optional[int] = None,
tol: Optional[float] = None,
) -> "GraphFrame":
"""
Runs the PageRank algorithm on the graph.
Note: Exactly one of fixed_num_iter or tolerance must be set.
See Scala documentation for more details.
:param resetProbability: Probability of resetting to a random vertex.
:param sourceId: (optional) the source vertex for a personalized PageRank.
:param maxIter: If set, the algorithm is run for a fixed number
of iterations. This may not be set if the `tol` parameter is set.
:param tol: If set, the algorithm is run until the given tolerance.
This may not be set if the `numIter` parameter is set.
:return: GraphFrame with new vertices column "pagerank" and new edges column "weight"
"""
return GraphFrame._from_impl(
self._impl.pageRank(
resetProbability=resetProbability,
sourceId=sourceId,
maxIter=maxIter,
tol=tol,
)
)
def parallelPersonalizedPageRank(
self,
resetProbability: float = 0.15,
sourceIds: Optional[list[Any]] = None,
maxIter: Optional[int] = None,
) -> "GraphFrame":
"""
Run the personalized PageRank algorithm on the graph,
from the provided list of sources in parallel for a fixed number of iterations.
See Scala documentation for more details.
:param resetProbability: Probability of resetting to a random vertex
:param sourceIds: the source vertices for a personalized PageRank
:param maxIter: the fixed number of iterations this algorithm runs
:return: GraphFrame with new vertices column "pageranks" and new edges column "weight"
"""
return GraphFrame._from_impl(
self._impl.parallelPersonalizedPageRank(
resetProbability=resetProbability, sourceIds=sourceIds, maxIter=maxIter
)
)
def shortestPaths(self, landmarks: list[Any]) -> DataFrame:
"""
Runs the shortest path algorithm from a set of landmark vertices in the graph.
See Scala documentation for more details.
:param landmarks: a set of one or more landmarks
:return: DataFrame with new vertices column "distances"
"""
return self._impl.shortestPaths(landmarks=landmarks)
def stronglyConnectedComponents(self, maxIter: int) -> DataFrame:
"""
Runs the strongly connected components algorithm on this graph.
See Scala documentation for more details.
:param maxIter: the number of iterations to run
:return: DataFrame with new vertex column "component"
"""
return self._impl.stronglyConnectedComponents(maxIter=maxIter)
def svdPlusPlus(
self,
rank: int = 10,
maxIter: int = 2,
minValue: float = 0.0,
maxValue: float = 5.0,
gamma1: float = 0.007,
gamma2: float = 0.007,
gamma6: float = 0.005,
gamma7: float = 0.015,
) -> tuple[DataFrame, float]:
"""
Runs the SVD++ algorithm.
See Scala documentation for more details.
:return: Tuple of DataFrame with new vertex columns storing learned model, and loss value
"""
return self._impl.svdPlusPlus(
rank=rank,
maxIter=maxIter,
minValue=minValue,
maxValue=maxValue,
gamma1=gamma1,
gamma2=gamma2,
gamma6=gamma6,
gamma7=gamma7,
)
def triangleCount(self) -> DataFrame:
"""
Counts the number of triangles passing through each vertex in this graph.
See Scala documentation for more details.
:return: DataFrame with new vertex column "count"
"""
return self._impl.triangleCount()
def powerIterationClustering(
self, k: int, maxIter: int, weightCol: Optional[str] = None
) -> DataFrame:
"""
Power Iteration Clustering (PIC), a scalable graph clustering algorithm developed by Lin and Cohen.
From the abstract: PIC finds a very low-dimensional embedding of a dataset using truncated power iteration
on a normalized pair-wise similarity matrix of the data.
:param k: the numbers of clusters to create
:param maxIter: param for maximum number of iterations (>= 0)
:param weightCol: optional name of weight column, 1.0 is used if not provided
:return: DataFrame with new column "cluster"
""" # noqa: E501
return self._impl.powerIterationClustering(k, maxIter, weightCol)
def _test():
import doctest
import graphframe
globs = graphframe.__dict__.copy()
globs["spark"] = SparkSession.builder.master("local[4]").appName("PythonTest").getOrCreate()
(failure_count, test_count) = doctest.testmod(
globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE
)
globs["spark"].stop()
if failure_count:
exit(-1)
if __name__ == "__main__":
_test()