aboutsummaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
authorAndrew Or <andrew@databricks.com>2016-05-03 13:47:58 -0700
committerReynold Xin <rxin@databricks.com>2016-05-03 13:47:58 -0700
commit588cac414a9cf1e0f40a82cc6a78f77e26825f29 (patch)
treeaeaa2ed8067942678341aaa599d63b173f4cfa42 /sql
parent83ee92f60345f016a390d61a82f1d924f64ddf90 (diff)
downloadspark-588cac414a9cf1e0f40a82cc6a78f77e26825f29.tar.gz
spark-588cac414a9cf1e0f40a82cc6a78f77e26825f29.tar.bz2
spark-588cac414a9cf1e0f40a82cc6a78f77e26825f29.zip
[SPARK-15073][SQL] Hide SparkSession constructor from the public
## What changes were proposed in this pull request? Users should use the builder pattern instead. ## How was this patch tested? Jenks. Author: Andrew Or <andrew@databricks.com> Closes #12873 from andrewor14/spark-session-constructor.
Diffstat (limited to 'sql')
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala14
1 files changed, 12 insertions, 2 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
index 3836ce2daa..aa7c335c53 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
@@ -54,6 +54,7 @@ import org.apache.spark.util.Utils
* {{{
* SparkSession.builder()
* .master("local")
+ * .appName("Word Count")
* .config("spark.some.config.option", "some-value").
* .getOrCreate()
* }}}
@@ -63,7 +64,7 @@ class SparkSession private(
@transient private val existingSharedState: Option[SharedState])
extends Serializable with Logging { self =>
- def this(sc: SparkContext) {
+ private[sql] def this(sc: SparkContext) {
this(sc, None)
}
@@ -573,7 +574,7 @@ class SparkSession private(
* common Scala objects into [[DataFrame]]s.
*
* {{{
- * val sparkSession = new SparkSession(sc)
+ * val sparkSession = SparkSession.builder.getOrCreate()
* import sparkSession.implicits._
* }}}
*
@@ -586,6 +587,15 @@ class SparkSession private(
}
// scalastyle:on
+ /**
+ * Stop the underlying [[SparkContext]].
+ *
+ * @since 2.0.0
+ */
+ def stop(): Unit = {
+ sparkContext.stop()
+ }
+
protected[sql] def parseSql(sql: String): LogicalPlan = {
sessionState.sqlParser.parsePlan(sql)
}