Skip to content

Commit a2c1dbc

Browse files
authored
Remove deprecated sqlContext in graphframe scala code (#432)
* update Signed-off-by: Weichen Xu <[email protected]> * update tests Signed-off-by: Weichen Xu <[email protected]> * update guide Signed-off-by: Weichen Xu <[email protected]> * update example Signed-off-by: Weichen Xu <[email protected]> --------- Signed-off-by: Weichen Xu <[email protected]>
1 parent 3d76998 commit a2c1dbc

19 files changed

+110
-113
lines changed

docs/quick-start.md

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -61,13 +61,13 @@ The following example shows how to create a GraphFrame, query it, and run the Pa
6161
// import graphframes package
6262
import org.graphframes._
6363
// Create a Vertex DataFrame with unique ID column "id"
64-
val v = sqlContext.createDataFrame(List(
64+
val v = spark.createDataFrame(List(
6565
("a", "Alice", 34),
6666
("b", "Bob", 36),
6767
("c", "Charlie", 30)
6868
)).toDF("id", "name", "age")
6969
// Create an Edge DataFrame with "src" and "dst" columns
70-
val e = sqlContext.createDataFrame(List(
70+
val e = spark.createDataFrame(List(
7171
("a", "b", "friend"),
7272
("b", "c", "follow"),
7373
("c", "b", "follow")
@@ -91,13 +91,13 @@ results.vertices.select("id", "pagerank").show()
9191
<div data-lang="python" markdown="1">
9292
{% highlight python %}
9393
# Create a Vertex DataFrame with unique ID column "id"
94-
v = sqlContext.createDataFrame([
94+
v = spark.createDataFrame([
9595
("a", "Alice", 34),
9696
("b", "Bob", 36),
9797
("c", "Charlie", 30),
9898
], ["id", "name", "age"])
9999
# Create an Edge DataFrame with "src" and "dst" columns
100-
e = sqlContext.createDataFrame([
100+
e = spark.createDataFrame([
101101
("a", "b", "friend"),
102102
("b", "c", "follow"),
103103
("c", "b", "follow"),

docs/user-guide.md

Lines changed: 23 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,7 @@ The following example demonstrates how to create a GraphFrame from vertex and ed
3737
{% highlight scala %}
3838
import org.graphframes.GraphFrame
3939
// Vertex DataFrame
40-
val v = sqlContext.createDataFrame(List(
40+
val v = spark.createDataFrame(List(
4141
("a", "Alice", 34),
4242
("b", "Bob", 36),
4343
("c", "Charlie", 30),
@@ -47,7 +47,7 @@ val v = sqlContext.createDataFrame(List(
4747
("g", "Gabby", 60)
4848
)).toDF("id", "name", "age")
4949
// Edge DataFrame
50-
val e = sqlContext.createDataFrame(List(
50+
val e = spark.createDataFrame(List(
5151
("a", "b", "friend"),
5252
("b", "c", "follow"),
5353
("c", "b", "follow"),
@@ -71,7 +71,7 @@ val g: GraphFrame = examples.Graphs.friends
7171
<div data-lang="python" markdown="1">
7272
{% highlight python %}
7373
# Vertex DataFrame
74-
v = sqlContext.createDataFrame([
74+
v = spark.createDataFrame([
7575
("a", "Alice", 34),
7676
("b", "Bob", 36),
7777
("c", "Charlie", 30),
@@ -81,7 +81,7 @@ v = sqlContext.createDataFrame([
8181
("g", "Gabby", 60)
8282
], ["id", "name", "age"])
8383
# Edge DataFrame
84-
e = sqlContext.createDataFrame([
84+
e = spark.createDataFrame([
8585
("a", "b", "friend"),
8686
("b", "c", "follow"),
8787
("c", "b", "follow"),
@@ -98,7 +98,7 @@ g = GraphFrame(v, e)
9898
The GraphFrame constructed above is available in the GraphFrames package:
9999
{% highlight python %}
100100
from graphframes.examples import Graphs
101-
g = Graphs(sqlContext).friends()
101+
g = Graphs.friends()
102102
{% endhighlight %}
103103
</div>
104104

@@ -166,7 +166,7 @@ val numFollows = g.edges.filter("relationship = 'follow'").count()
166166
<div data-lang="python" markdown="1">
167167
{% highlight python %}
168168
from graphframes.examples import Graphs
169-
g = Graphs(sqlContext).friends() # Get example graph
169+
g = Graphs.friends() # Get example graph
170170

171171
# Display the vertex and edge DataFrames
172172
g.vertices.show()
@@ -294,7 +294,7 @@ For API details, refer to the [API docs](api/python/graphframes.html#graphframes
294294

295295
{% highlight python %}
296296
from graphframes.examples import Graphs
297-
g = Graphs(sqlContext).friends() # Get example graph
297+
g = Graphs.friends() # Get example graph
298298

299299
# Search for pairs of vertices with edges in both directions between them.
300300
motifs = g.find("(a)-[e]->(b); (b)-[e2]->(a)")
@@ -360,7 +360,7 @@ chainWith2Friends2.show()
360360
from pyspark.sql.functions import col, lit, when
361361
from pyspark.sql.types import IntegerType
362362
from graphframes.examples import Graphs
363-
g = Graphs(sqlContext).friends() # Get example graph
363+
g = Graphs.friends() # Get example graph
364364

365365
chain4 = g.find("(a)-[ab]->(b); (b)-[bc]->(c); (c)-[cd]->(d)")
366366

@@ -414,7 +414,7 @@ val g1 = g.filterVertices("age > 30").filterEdges("relationship = 'friend'").dro
414414
<div data-lang="python" markdown="1">
415415
{% highlight python %}
416416
from graphframes.examples import Graphs
417-
g = Graphs(sqlContext).friends() # Get example graph
417+
g = Graphs.friends() # Get example graph
418418

419419
# Select subgraph of users older than 30, and relationships of type "friend".
420420
# Drop isolated vertices (users) which are not contained in any edges (relationships).
@@ -455,7 +455,7 @@ val g2 = GraphFrame(g.vertices, e2)
455455
<div data-lang="python" markdown="1">
456456
{% highlight python %}
457457
from graphframes.examples import Graphs
458-
g = Graphs(sqlContext).friends() # Get example graph
458+
g = Graphs.friends() # Get example graph
459459

460460
# Select subgraph based on edges "e" of type "follow"
461461
# pointing from a younger user "a" to an older user "b".
@@ -522,7 +522,7 @@ For API details, refer to the [API docs](api/python/graphframes.html#graphframes
522522

523523
{% highlight python %}
524524
from graphframes.examples import Graphs
525-
g = Graphs(sqlContext).friends() # Get example graph
525+
g = Graphs.friends() # Get example graph
526526

527527
# Search from "Esther" for users of age < 32.
528528
paths = g.bfs("name = 'Esther'", "age < 32")
@@ -568,7 +568,7 @@ For API details, refer to the [API docs](api/python/graphframes.html#graphframes
568568

569569
{% highlight python %}
570570
from graphframes.examples import Graphs
571-
g = Graphs(sqlContext).friends() # Get example graph
571+
g = Graphs.friends() # Get example graph
572572

573573
result = g.connectedComponents()
574574
result.select("id", "component").orderBy("component").show()
@@ -605,7 +605,7 @@ For API details, refer to the [API docs](api/python/graphframes.html#graphframes
605605

606606
{% highlight python %}
607607
from graphframes.examples import Graphs
608-
g = Graphs(sqlContext).friends() # Get example graph
608+
g = Graphs.friends() # Get example graph
609609

610610
result = g.stronglyConnectedComponents(maxIter=10)
611611
result.select("id", "component").orderBy("component").show()
@@ -649,7 +649,7 @@ For API details, refer to the [API docs](api/python/graphframes.html#graphframes
649649

650650
{% highlight python %}
651651
from graphframes.examples import Graphs
652-
g = Graphs(sqlContext).friends() # Get example graph
652+
g = Graphs.friends() # Get example graph
653653

654654
result = g.labelPropagation(maxIter=5)
655655
result.select("id", "label").show()
@@ -707,7 +707,7 @@ For API details, refer to the [API docs](api/python/graphframes.html#graphframes
707707

708708
{% highlight python %}
709709
from graphframes.examples import Graphs
710-
g = Graphs(sqlContext).friends() # Get example graph
710+
g = Graphs.friends() # Get example graph
711711

712712
# Run PageRank until convergence to tolerance "tol".
713713
results = g.pageRank(resetProbability=0.15, tol=0.01)
@@ -759,7 +759,7 @@ For API details, refer to the [API docs](api/python/graphframes.html#graphframes
759759

760760
{% highlight python %}
761761
from graphframes.examples import Graphs
762-
g = Graphs(sqlContext).friends() # Get example graph
762+
g = Graphs.friends() # Get example graph
763763

764764
results = g.shortestPaths(landmarks=["a", "d"])
765765
results.select("id", "distances").show()
@@ -793,7 +793,7 @@ For API details, refer to the [API docs](api/python/graphframes.html#graphframes
793793

794794
{% highlight python %}
795795
from graphframes.examples import Graphs
796-
g = Graphs(sqlContext).friends() # Get example graph
796+
g = Graphs.friends() # Get example graph
797797

798798
results = g.triangleCount()
799799
results.select("id", "count").show()
@@ -823,8 +823,8 @@ g.vertices.write.parquet("hdfs://myLocation/vertices")
823823
g.edges.write.parquet("hdfs://myLocation/edges")
824824

825825
// Load the vertices and edges back.
826-
val sameV = sqlContext.read.parquet("hdfs://myLocation/vertices")
827-
val sameE = sqlContext.read.parquet("hdfs://myLocation/edges")
826+
val sameV = spark.read.parquet("hdfs://myLocation/vertices")
827+
val sameE = spark.read.parquet("hdfs://myLocation/edges")
828828

829829
// Create an identical GraphFrame.
830830
val sameG = GraphFrame(sameV, sameE)
@@ -834,15 +834,15 @@ val sameG = GraphFrame(sameV, sameE)
834834
<div data-lang="python" markdown="1">
835835
{% highlight python %}
836836
from graphframes.examples import Graphs
837-
g = Graphs(sqlContext).friends() # Get example graph
837+
g = Graphs.friends() # Get example graph
838838

839839
# Save vertices and edges as Parquet to some location.
840840
g.vertices.write.parquet("hdfs://myLocation/vertices")
841841
g.edges.write.parquet("hdfs://myLocation/edges")
842842

843843
# Load the vertices and edges back.
844-
sameV = sqlContext.read.parquet("hdfs://myLocation/vertices")
845-
sameE = sqlContext.read.parquet("hdfs://myLocation/edges")
844+
sameV = spark.read.parquet("hdfs://myLocation/vertices")
845+
sameE = spark.read.parquet("hdfs://myLocation/edges")
846846

847847
# Create an identical GraphFrame.
848848
sameG = GraphFrame(sameV, sameE)
@@ -903,7 +903,7 @@ For API details, refer to the
903903
from pyspark.sql.functions import sum as sqlsum
904904
from graphframes.lib import AggregateMessages as AM
905905
from graphframes.examples import Graphs
906-
g = Graphs(sqlContext).friends() # Get example graph
906+
g = Graphs.friends() # Get example graph
907907

908908
# For each user, sum the ages of the adjacent users.
909909
msgToSrc = AM.dst["age"]

src/main/scala/org/graphframes/GraphFrame.scala

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -507,12 +507,12 @@ class GraphFrame private(
507507
val (nextDF, nextNames) = findIncremental(this, handledPatterns, dfOpt, names, cur)
508508
(handledPatterns :+ cur, nextDF, nextNames)
509509
}
510-
finalDFOpt.getOrElse(sqlContext.emptyDataFrame)
510+
finalDFOpt.getOrElse(spark.emptyDataFrame)
511511
}
512512

513513
// ========= Other private methods ===========
514514

515-
private[graphframes] def sqlContext: SQLContext = vertices.sqlContext
515+
private[graphframes] def spark: SparkSession = vertices.sparkSession
516516

517517
/**
518518
* True if the id type can be cast to Long.
@@ -618,8 +618,8 @@ object GraphFrame extends Serializable with Logging {
618618
joinCol: String,
619619
hubs: Set[T],
620620
logPrefix: String): DataFrame = {
621-
val sqlContext = a.sqlContext
622-
import sqlContext.implicits._
621+
val spark = a.sparkSession
622+
import spark.implicits._
623623
if (hubs.isEmpty) {
624624
// No skew. Do regular join.
625625
a.join(b, joinCol)
@@ -719,9 +719,9 @@ object GraphFrame extends Serializable with Logging {
719719
*/
720720
// TODO: Add version which takes explicit schemas.
721721
def fromGraphX[VD : TypeTag, ED : TypeTag](graph: Graph[VD, ED]): GraphFrame = {
722-
val sqlContext = SparkSession.builder().getOrCreate().sqlContext
723-
val vv = sqlContext.createDataFrame(graph.vertices).toDF(ID, ATTR)
724-
val ee = sqlContext.createDataFrame(graph.edges).toDF(SRC, DST, ATTR)
722+
val spark = SparkSession.builder().getOrCreate()
723+
val vv = spark.createDataFrame(graph.vertices).toDF(ID, ATTR)
724+
val ee = spark.createDataFrame(graph.edges).toDF(SRC, DST, ATTR)
725725
GraphFrame(vv, ee)
726726
}
727727

src/main/scala/org/graphframes/examples/BeliefPropagation.scala

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ package org.graphframes.examples
1919

2020
import org.apache.spark.{SparkConf, SparkContext}
2121
import org.apache.spark.graphx.{Graph, VertexRDD, Edge => GXEdge}
22-
import org.apache.spark.sql.{Column, Row, SparkSession, SQLContext}
22+
import org.apache.spark.sql.{Column, Row, SparkSession}
2323
import org.apache.spark.sql.functions.{col, lit, sum, udf, when}
2424

2525
import org.graphframes.GraphFrame
@@ -74,10 +74,8 @@ object BeliefPropagation {
7474
.appName("BeliefPropagation example")
7575
.getOrCreate()
7676

77-
val sql = spark.sqlContext
78-
7977
// Create graphical model g of size 3 x 3.
80-
val g = gridIsingModel(sql, 3)
78+
val g = gridIsingModel(spark, 3)
8179

8280
println("Original Ising model:")
8381
g.vertices.show()

0 commit comments

Comments
 (0)