-
Notifications
You must be signed in to change notification settings - Fork 264
Expand file tree
/
Copy pathgraphframe.py
More file actions
1062 lines (907 loc) · 45.6 KB
/
graphframe.py
File metadata and controls
1062 lines (907 loc) · 45.6 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
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
#
# 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
import warnings
from typing import TYPE_CHECKING, Any, final
from pyspark.sql import functions as F
from pyspark.storagelevel import StorageLevel
from pyspark.version import __version__
try:
from typing import override
except ImportError:
from typing_extensions import override
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 graphframes.classic.graphframe import GraphFrame as GraphFrameClassic
from graphframes.internal.utils import (
_HASH2VEC_DECAY_FUNCTIONS,
_RandomWalksEmbeddingsParameters,
)
from graphframes.lib import Pregel
if TYPE_CHECKING:
from pyspark.sql import Column, DataFrame
from graphframes.connect.graphframes_client import GraphFrameConnect
"""Constant for the vertices ID column name."""
ID = "id"
"""Constant for the edge src column name."""
SRC = "src"
"""Constant for the edge dst column name."""
DST = "dst"
"""Constant for the edge column name."""
EDGE = "edge"
"""Constant for the weight column name."""
WEIGHT = "weight"
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)
"""
ID: str = ID
SRC: str = SRC
DST: str = DST
EDGE: str = EDGE
WEIGHT: str = WEIGHT
@staticmethod
def _from_impl(impl: "GraphFrameClassic | GraphFrameConnect") -> "GraphFrame":
return GraphFrame(impl._vertices, impl._edges)
def __init__(self, v: DataFrame, e: DataFrame) -> None:
"""
Initialize a GraphFrame from vertex DataFrame and edges DataFrame.
: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.
"""
self._impl: "GraphFrameClassic | GraphFrameConnect"
if self.ID not in v.columns:
raise ValueError(
"Vertex ID column {} missing from vertex DataFrame, which has columns: {}".format(
self.ID, ",".join(v.columns)
)
)
if self.SRC not in e.columns:
raise ValueError(
"Source vertex ID column {} missing from edge DataFrame, which has columns: {}".format( # noqa: E501
self.SRC, ",".join(e.columns)
)
)
if self.DST not in e.columns:
raise ValueError(
"Destination vertex ID column {} missing from edge DataFrame, which has columns: {}".format( # noqa: E501
self.DST, ",".join(e.columns)
)
)
if is_remote():
from graphframes.connect.graphframes_client import GraphFrameConnect
self._impl = GraphFrameConnect(v, e) # ty: ignore[invalid-argument-type]
else:
self._impl = GraphFrameClassic(v, e) # ty: ignore[invalid-argument-type]
@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
@property
def nodes(self) -> DataFrame:
"""Alias to vertices."""
return self.vertices
@override
def __repr__(self) -> str:
# Exactly like in the scala core
v_cols = [self.ID] + [col for col in self._impl._vertices.columns if col != self.ID]
e_cols = [self.SRC, self.DST] + [
col for col in self._impl._edges.columns if col not in {self.SRC, self.DST}
]
v = self._impl._vertices.select(*v_cols).__repr__()
e = self._impl._edges.select(*e_cols).__repr__()
return f"GraphFrame(v:{v}, e:{e})"
def cache(self) -> "GraphFrame":
"""Persist the dataframe representation of vertices and edges of the graph with the default
storage level.
"""
new_vertices = self._impl._vertices.cache()
new_edges = self._impl._edges.cache()
return GraphFrame(new_vertices, new_edges)
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.
"""
new_vertices = self._impl._vertices.persist(storageLevel=storageLevel)
new_edges = self._impl._edges.persist(storageLevel=storageLevel)
return GraphFrame(new_vertices, new_edges)
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.
"""
new_vertices = self._impl._vertices.unpersist(blocking=blocking)
new_edges = self._impl._edges.unpersist(blocking=blocking)
return GraphFrame(new_vertices, new_edges)
@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._edges.groupBy(F.col(self.SRC).alias(self.ID)).agg(
F.count("*").alias("outDegree")
)
@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._edges.groupBy(F.col(self.DST).alias(self.ID)).agg(
F.count("*").alias("inDegree")
)
@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._edges.select(
F.explode(F.array(F.col(self.SRC), F.col(self.DST))).alias(self.ID)
)
.groupBy(self.ID)
.agg(F.count("*").alias("degree"))
)
def type_out_degree(self, edge_type_col: str, edge_types: list[Any] | None = None) -> DataFrame:
"""
The out-degree of each vertex per edge type, returned as a DataFrame with two columns:
- "id": the ID of the vertex
- "outDegrees": a struct with a field for each edge type, storing the out-degree count
:param edge_type_col: Name of the column in edges DataFrame that contains edge types
:param edge_types: Optional list of edge type values. If None, edge types will be
discovered automatically.
:return: DataFrame with columns "id" and "outDegrees" (struct type)
"""
if edge_types is not None:
pivot_df = self._impl._edges.groupBy(F.col(self.SRC).alias(self.ID)).pivot(
edge_type_col, edge_types
)
else:
pivot_df = self._impl._edges.groupBy(F.col(self.SRC).alias(self.ID)).pivot(
edge_type_col
)
count_df = pivot_df.agg(F.count(F.lit(1))).na.fill(0)
struct_cols = [
F.col(col_name).cast("int").alias(col_name)
for col_name in count_df.columns
if col_name != self.ID
]
return count_df.select(F.col(self.ID), F.struct(*struct_cols).alias("outDegrees"))
def type_in_degree(self, edge_type_col: str, edge_types: list[Any] | None = None) -> DataFrame:
"""
The in-degree of each vertex per edge type, returned as a DataFrame with two columns:
- "id": the ID of the vertex
- "inDegrees": a struct with a field for each edge type, storing the in-degree count
:param edge_type_col: Name of the column in edges DataFrame that contains edge types
:param edge_types: Optional list of edge type values. If None, edge types will be
discovered automatically.
:return: DataFrame with columns "id" and "inDegrees" (struct type)
"""
if edge_types is not None:
pivot_df = self._impl._edges.groupBy(F.col(self.DST).alias(self.ID)).pivot(
edge_type_col, edge_types
)
else:
pivot_df = self._impl._edges.groupBy(F.col(self.DST).alias(self.ID)).pivot(
edge_type_col
)
count_df = pivot_df.agg(F.count(F.lit(1))).na.fill(0)
struct_cols = [
F.col(col_name).cast("int").alias(col_name)
for col_name in count_df.columns
if col_name != self.ID
]
return count_df.select(F.col(self.ID), F.struct(*struct_cols).alias("inDegrees"))
def type_degree(self, edge_type_col: str, edge_types: list[Any] | None = None) -> DataFrame:
"""
The total degree of each vertex per edge type (both in and out), returned as a DataFrame
with two columns:
- "id": the ID of the vertex
- "degrees": a struct with a field for each edge type, storing the total degree count
:param edge_type_col: Name of the column in edges DataFrame that contains edge types
:param edge_types: Optional list of edge type values. If None, edge types will be
discovered automatically.
:return: DataFrame with columns "id" and "degrees" (struct type)
"""
exploded_edges = self._impl._edges.select(
F.explode(F.array(F.col(self.SRC), F.col(self.DST))).alias(self.ID),
F.col(edge_type_col),
)
if edge_types is not None:
pivot_df = exploded_edges.groupBy(self.ID).pivot(edge_type_col, edge_types)
else:
pivot_df = exploded_edges.groupBy(self.ID).pivot(edge_type_col)
count_df = pivot_df.agg(F.count(F.lit(1))).na.fill(0)
struct_cols = [
F.col(col_name).cast("int").alias(col_name)
for col_name in count_df.columns
if col_name != self.ID
]
return count_df.select(F.col(self.ID), F.struct(*struct_cols).alias("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.classic.pregel.Pregel`
or :class`graphframes.connect.graphframes_client.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 detectingCycles(
self,
checkpoint_interval: int = 2,
use_local_checkpoints: bool = False,
storage_level: StorageLevel = StorageLevel.MEMORY_AND_DISK_DESER,
) -> DataFrame:
"""Find all cycles in the graph.
An implementation of the Rocha–Thatte cycle detection algorithm.
Rocha, Rodrigo Caetano, and Bhalchandra D. Thatte. "Distributed cycle detection in
large-scale sparse graphs." Proceedings of Simpósio Brasileiro de Pesquisa Operacional
(SBPO'15) (2015): 1-11.
Returns a DataFrame with unique cycles.
:param checkpoint_interval: Pregel checkpoint interval, default is 2
:param use_local_checkpoints: should local checkpoints be used instead of checkpointDir
:storage_level: the level of storage for both intermediate results and an output DataFrame
:return: Persisted DataFrame with all the cycles
"""
return self._impl.detectingCycles(checkpoint_interval, use_local_checkpoints, storage_level)
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: list[Column | str] | Column,
sendToSrc: list[Column | str] | Column | str | None = None,
sendToDst: list[Column | str] | Column | str | None = None,
intermediate_storage_level: StorageLevel = StorageLevel.MEMORY_AND_DISK_DESER,
) -> 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.
Warning! The result of this method is persisted DataFrame object! Users should handle unpersist
to avoid possible memory leaks!
:param aggCol: the requested aggregation output either as a collection of
:class:`pyspark.sql.Column` or SQL expression string
:param sendToSrc: message sent to the source vertex of each triplet either as
a collection of :class:`pyspark.sql.Column` or SQL expression string (default: None)
:param sendToDst: message sent to the destination vertex of each triplet either as
collection of :class:`pyspark.sql.Column` or SQL expression string (default: None)
:param intermediate_storage_level: the level of intermediate storage that will be used
for both intermediate result and the output.
:return: Persisted DataFrame with columns for the vertex ID and the resulting aggregated message.
The name of the resulted message column is based on the alias of the provided aggCol!
""" # noqa: E501
if sendToDst is None:
sendToDst = []
if sendToSrc is None:
sendToSrc = []
# Back-compatibility workaround
if not isinstance(aggCol, list):
warnings.warn(
"Passing single column to aggCol is deprecated, use list",
DeprecationWarning,
)
return self.aggregateMessages(
[aggCol], sendToSrc, sendToDst, intermediate_storage_level
)
if not isinstance(sendToSrc, list):
warnings.warn(
"Passing single column to sendToSrc is deprecated, use list",
DeprecationWarning,
)
return self.aggregateMessages(
aggCol, [sendToSrc], sendToDst, intermediate_storage_level
)
if not isinstance(sendToDst, list):
warnings.warn(
"Passing single column to sendToDst is deprecated, use list",
DeprecationWarning,
)
return self.aggregateMessages(
aggCol, sendToSrc, [sendToDst], intermediate_storage_level
)
if len(aggCol) == 0:
raise TypeError("At least one aggregation column should be provided!")
if (len(sendToSrc) == 0) and (len(sendToDst) == 0):
raise ValueError("Either `sendToSrc`, `sendToDst`, or both have to be provided")
return self._impl.aggregateMessages(
aggCol=aggCol,
sendToSrc=sendToSrc,
sendToDst=sendToDst,
intermediate_storage_level=intermediate_storage_level,
)
# Standard algorithms
def connectedComponents(
self,
algorithm: str = "graphframes",
checkpointInterval: int = 2,
broadcastThreshold: int = 1000000,
useLabelsAsComponents: bool = False,
use_local_checkpoints: bool = False,
max_iter: int = 2 ^ 31 - 2,
storage_level: StorageLevel = StorageLevel.MEMORY_AND_DISK_DESER,
) -> 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 "two_phase", "randomized_contraction",
"graphframes" (deprecated alias for "two_phase") and "graphx".
:param checkpointInterval: checkpoint interval in terms of number of iterations (default: 2)
:param broadcastThreshold: broadcast threshold in propagating component assignments
(default: 1000000). Passing -1 disable manual broadcasting and
allows AQE to handle skewed joins. This mode is much faster
and is recommended to use. Default value may be changed to -1
in the future versions of GraphFrames.
:param useLabelsAsComponents: if True, uses the vertex labels as components, otherwise will
use longs
:param use_local_checkpoints: should local checkpoints be used, default false;
local checkpoints are faster and does not require to set
a persistent checkpointDir; from the other side, local
checkpoints are less reliable and require executors to have
big enough local disks.
:param storage_level: storage level for both intermediate and final dataframes.
:return: DataFrame with new vertices column "component"
"""
return self._impl.connectedComponents(
algorithm=algorithm,
checkpointInterval=checkpointInterval,
broadcastThreshold=broadcastThreshold,
useLabelsAsComponents=useLabelsAsComponents,
use_local_checkpoints=use_local_checkpoints,
max_iter=max_iter,
storage_level=storage_level,
)
def maximal_independent_set(
self,
seed: int = 42,
checkpoint_interval: int = 2,
use_local_checkpoints: bool = False,
storage_level: StorageLevel = StorageLevel.MEMORY_AND_DISK_DESER,
) -> DataFrame:
"""
This method implements a distributed algorithm for finding a Maximal Independent Set (MIS)
in a graph.
An MIS is a set of vertices such that no two vertices in the set are adjacent (i.e., there
is no edge between any two vertices in the set), and the set is maximal, meaning that adding
any other vertex to the set would violate the independence property. Note that this
implementation finds a maximal (but not necessarily maximum) independent set; that is, it
ensures no more vertices can be added to the set, but does not guarantee that the set has
the largest possible number of vertices among all possible independent sets in the graph.
The algorithm implemented here is based on the paper: Ghaffari, Mohsen. "An improved
distributed algorithm for maximal independent set." Proceedings of the twenty-seventh annual
ACM-SIAM symposium on Discrete algorithms. Society for Industrial and Applied Mathematics,
2016.
Note: This is a randomized, non-deterministic algorithm. The result may vary between runs
even if a fixed random seed is provided because of how Apache Spark works.
:param seed: random seed used for tie-breaking in the algorithm (default: 42)
:param checkpoint_interval: checkpoint interval in terms of number of iterations (default: 2)
:param use_local_checkpoints: whether to use local checkpoints (default: False);
local checkpoints are faster and do not require setting
a persistent checkpoint directory; however, they are less
reliable and require executors to have sufficient local disk space.
:param storage_level: storage level for both intermediate and final DataFrames
(default: MEMORY_AND_DISK_DESER)
:return: DataFrame with new vertex column "selected", where "true" indicates the vertex
is part of the Maximal Independent Set
""" # noqa: E501
return self._impl.maximal_independent_set(
checkpoint_interval, storage_level, use_local_checkpoints, seed
)
def k_core(
self,
checkpoint_interval: int = 2,
use_local_checkpoints: bool = False,
storage_level: StorageLevel = StorageLevel.MEMORY_AND_DISK_DESER,
) -> DataFrame:
"""
The k-core is the maximal subgraph such that every vertex has at least degree k.
The k-core metric is a measure of the centrality of a node in a network, based on its
degree and the degrees of its neighbors. Nodes with higher k-core values are considered
to be more central and influential within the network.
This implementation is based on the algorithm described in:
Mandal, Aritra, and Mohammad Al Hasan. "A distributed k-core decomposition algorithm
on spark." 2017 IEEE International Conference on Big Data (Big Data). IEEE, 2017.
:param checkpoint_interval: Pregel checkpoint interval, default is 2
:param use_local_checkpoints: should local checkpoints be used instead of checkpointDir
:param storage_level: the level of storage for both intermediate results and an output DataFrame
:return: Persisted DataFrame with ID and k-core values (column "kcore")
""" # noqa: E501
return self._impl.k_core(checkpoint_interval, use_local_checkpoints, storage_level)
def labelPropagation(
self,
maxIter: int,
algorithm: str = "graphx",
use_local_checkpoints: bool = False,
checkpoint_interval: int = 2,
storage_level: StorageLevel = StorageLevel.MEMORY_AND_DISK_DESER,
) -> 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
:param algorithm: implementation to use, posible values are "graphframes" and "graphx";
"graphx" is faster for small-medium sized graphs,
"graphframes" requires less amount of memory
:param use_local_checkpoints: should local checkpoints be used, default false;
local checkpoints are faster and does not require to set
a persistent checkpointDir; from the other side, local
checkpoints are less reliable and require executors to have
big enough local disks.
:checkpoint_interval: How often should the intermediate result be checkpointed;
Using big value here may tend to huge logical plan growth due
to the iterative nature of the algorithm.
:param storage_level: storage level for both intermediate and final dataframes.
:return: Persisted DataFrame with new vertices column "label"
"""
return self._impl.labelPropagation(
maxIter=maxIter,
algorithm=algorithm,
use_local_checkpoints=use_local_checkpoints,
checkpoint_interval=checkpoint_interval,
storage_level=storage_level,
)
def pageRank(
self,
resetProbability: float = 0.15,
sourceId: Any | None = None,
maxIter: int | None = None,
tol: float | None = 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: list[Any] | None = None,
maxIter: int | None = 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[str | int],
algorithm: str = "graphx",
use_local_checkpoints: bool = False,
checkpoint_interval: int = 2,
storage_level: StorageLevel = StorageLevel.MEMORY_AND_DISK_DESER,
is_directed: bool = True,
) -> 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
:param algorithm: implementation to use, posible values are "graphframes" and "graphx";
"graphx" is faster for small-medium sized graphs,
"graphframes" requires less amount of memory
:param use_local_checkpoints: should local checkpoints be used, default false;
local checkpoints are faster and does not require to set
a persistent checkpointDir; from the other side, local
checkpoints are less reliable and require executors to have
big enough local disks.
:param checkpoint_interval: How often should the intermediate result be checkpointed;
Using big value here may tend to huge logical plan growth due
to the iterative nature of the algorithm.
:param storage_level: storage level for both intermediate and final dataframes.
:param is_directed: should algorithm find directed paths or any paths.
:return: persistent DataFrame with new vertices column "distances"
""" # noqa: E501
return self._impl.shortestPaths(
landmarks=landmarks,
algorithm=algorithm,
use_local_checkpoints=use_local_checkpoints,
checkpoint_interval=checkpoint_interval,
storage_level=storage_level,
is_directed=is_directed,
)
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 for Collaborative Filtering.
Based on the paper "Factorization Meets the Neighborhood: a Multifaceted Collaborative
Filtering Model" by Yehuda Koren (2008).
**Algorithm Description**
SVD++ improves upon standard Matrix Factorization by incorporating implicit feedback
(the history of items a user has interacted with) alongside explicit ratings.
The prediction rule is:
``r_ui = µ + b_u + b_i + q_i^T * (p_u + |N(u)|^-0.5 * sum(y_j for j in N(u)))``
**Input Requirements**
The input graph must be a **Directed Bipartite Graph**:
- **Vertices**: A mix of Users and Items.
- **Edges**: Directed strictly from **User (src) -> Item (dst)**.
- **Edge Attribute**: Represents the rating (weight).
:param rank: The number of latent factors (embedding size).
:param maxIter: The maximum number of iterations.
:param minValue: The minimum possible rating value (used for clipping predictions).
:param maxValue: The maximum possible rating value (used for clipping predictions).
:param gamma1: Learning rate for bias parameters (`b_u`, `b_i`).
:param gamma2: Learning rate for factor parameters (`p_u`, `q_i`, `y_j`).
:param gamma6: Regularization coefficient for bias parameters.
:param gamma7: Regularization coefficient for factor parameters.
:return: A tuple ``(v, loss)`` where:
- ``v`` is a DataFrame of vertices containing the trained model parameters (embeddings).
- ``loss`` is the final training loss (double).
**Output DataFrame Columns**
The returned DataFrame ``v`` contains the following new columns containing the model parameters:
- **column1** (Array[Double]): Primary Latent Factors (Explicit Embedding).
- For Users: Preferences vector (`p_u`).
- For Items: Characteristics vector (`q_i`).
- **column2** (Array[Double]): Implicit Factors (Implicit Embedding).
- For Items: Influence vector (`y_i`).
- For Users: Unused/Zero (users aggregate `y` from neighbors).
- **column3** (Double): Bias term.
- For Users: User bias (`b_u`).
- For Items: Item bias (`b_i`).
- **column4** (Double): Implicit Normalization term.
- For Users: Precomputed ``|N(u)|^-0.5``.
- For Items: Unused.
""" # noqa: E501
return self._impl.svdPlusPlus(
rank=rank,
maxIter=maxIter,
minValue=minValue,
maxValue=maxValue,
gamma1=gamma1,
gamma2=gamma2,
gamma6=gamma6,
gamma7=gamma7,
)
def triangleCount(
self, storage_level: StorageLevel, algorithm: str = "exact", lg_nom_entries: int = 12
) -> DataFrame:
"""
Computes the number of triangles passing through each vertex.
This algorithm identifies sets of three vertices where each pair is connected by an edge.
The implementation provides two algorithms:
- "exact": Computes the exact triangle count using set intersection of neighbor lists.
Note: This method can fail or encounter OOM errors on power-law graphs or graphs with
very high-degree nodes, as it requires collecting and intersecting the full neighbor
lists for the source and destination vertices of every edge.
- "approx": Uses DataSketches (Theta sketches) to estimate the triangle count. This
trades off perfect accuracy for significantly improved performance and lower memory
overhead, making it suitable for large-scale or dense graphs.
:param storage_level: Storage level for caching intermediate DataFrames.
:param algorithm: The triangle counting algorithm to use, "exact" or "approx" (default: "exact").
:param lg_nom_entries: The log2 of the nominal entries for the Theta sketch (only used
if algorithm="approx"). Higher values increase accuracy at the
cost of memory. (default: 12).
:return: A DataFrame containing the vertex "id" and the triangle "count".
""" # noqa: E501
if (__version__[:3] < "4.1") and (algorithm == "approx"):
raise ValueError("approximate algorithm requires Spark 4.1+")
return self._impl.triangleCount(
storage_level=storage_level, algorithm=algorithm, log_nom_entries=lg_nom_entries
)
def powerIterationClustering(
self, k: int, maxIter: int, weightCol: str | None = 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 validate(
self,
check_vertices: bool = True,
intermediate_storage_level: StorageLevel = StorageLevel.MEMORY_AND_DISK_DESER,
) -> None:
"""
Validates the consistency and integrity of a graph by performing checks on the vertices and
edges.
:param check_vertices: a flag to indicate whether additional vertex consistency checks
should be performed. If true, the method will verify that all vertices in the vertex
DataFrame are represented in the edge DataFrame and vice versa. It is slow on big graphs.
:param intermediate_storage_level: the storage level to be used when persisting
intermediate DataFrame computations during the validation process.
:return: Unit, as the method performs validation checks and throws an exception if
validation fails.
:raises ValueError: if there are any inconsistencies in the graph, such as duplicate
vertices, mismatched vertices between edges and vertex DataFrames or missing
connections.
""" # noqa: E501
persisted_vertices = self.vertices.persist(intermediate_storage_level)
row = persisted_vertices.select(F.count_distinct(F.col(ID))).first()
assert row is not None # for type checker
count_distinct_vertices = row[0]
assert isinstance(count_distinct_vertices, int) # for type checker
total_count_vertices = persisted_vertices.count()
if count_distinct_vertices != total_count_vertices:
_msg = "Graph contains ({}) duplicate vertices."
raise ValueError(_msg.format(total_count_vertices - count_distinct_vertices))
if check_vertices:
vertices_set_from_edges = (
self.edges.select(F.col(SRC).alias(ID))
.union(self.edges.select(F.col(DST).alias(ID)))
.distinct()
.persist(intermediate_storage_level)
)
count_vertices_from_edges = vertices_set_from_edges.count()
if count_vertices_from_edges > count_distinct_vertices:
_msg = "Graph is inconsistent: edges has {} "
_msg += "vertices, but vertices has {} vertices."
raise ValueError(_msg.format(count_vertices_from_edges, count_distinct_vertices))
combined = vertices_set_from_edges.join(self.vertices, ID, "left_anti")
count_of_bad_vertices = combined.count()
if count_of_bad_vertices > 0:
_msg = "Vertices DataFrame does not contain all edges src/dst. "
_msg += "Found {} edges src/dst that are not in the vertices DataFrame."
raise ValueError(_msg.format(count_of_bad_vertices))
_ = vertices_set_from_edges.unpersist()
_ = persisted_vertices.unpersist()
def as_undirected(self) -> "GraphFrame":
"""
Converts the directed graph into an undirected graph by ensuring that all directed edges are
bidirectional. For every directed edge (src, dst), a corresponding edge (dst, src) is added.
:return: A new GraphFrame representing the undirected graph.
"""
edge_attr_columns = [c for c in self.edges.columns if c not in [SRC, DST]]
# Create the undirected edges by duplicating each edge in both directions
# 3.5.x problem: selecting empty struct fails on spark connect
# TODO: remove after removing 3.5.x
if edge_attr_columns:
forward_edges = self.edges.select(
F.col(SRC), F.col(DST), F.struct(*edge_attr_columns).alias(EDGE)
)
backward_edges = self.edges.select(
F.col(DST).alias(SRC),
F.col(SRC).alias(DST),
F.struct(*edge_attr_columns).alias(EDGE),
)
new_edges = forward_edges.union(backward_edges).select(SRC, DST, EDGE)
else:
forward_edges = self.edges.select(F.col(SRC), F.col(DST))
backward_edges = self.edges.select(F.col(DST).alias(SRC), F.col(SRC).alias(DST))
new_edges = forward_edges.union(backward_edges).select(SRC, DST)
# Preserve additional edge attributes
edge_columns = [F.col(EDGE).getField(c).alias(c) for c in edge_attr_columns]
# Select all columns including the new edge attributes
selected_columns = [F.col(SRC), F.col(DST)] + edge_columns
new_edges = new_edges.select(*selected_columns)
return GraphFrame(self.vertices, new_edges)
@final
class RandomWalkEmbeddings:
def __init__(self, graph: GraphFrame) -> None:
self._graph: GraphFrame = graph
self._params: _RandomWalksEmbeddingsParameters = _RandomWalksEmbeddingsParameters()
def use_cached_random_walks(self, cached_walks_path: str) -> None:
if cached_walks_path == "":
raise ValueError("cached walks path cannot be empty")
self._params.rw_cached_walks = cached_walks_path
def set_rw_model(
self,
temporary_prefix: str,
use_edge_direction: bool = False,
max_neighbors_per_vertex: int = 50,
num_walks_per_node: int = 5,
num_batches: int = 5,
walks_per_batch: int = 10,
restart_probability: float = 0.1,
seed: int = 42,
) -> None:
self._params.rw_model = "rw_with_restart"
self._params.rw_temporary_prefix = temporary_prefix
self._params.use_edge_direction = use_edge_direction
self._params.rw_max_nbrs = max_neighbors_per_vertex
self._params.rw_num_walks_per_node = num_walks_per_node
self._params.rw_num_batches = num_batches
self._params.rw_batch_size = walks_per_batch
self._params.rw_restart_probability = restart_probability
self._params.rw_seed = seed
def set_hash2vec(