[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.
This commit is contained in:
parent
83ee92f603
commit
588cac414a
|
@ -18,7 +18,6 @@
|
|||
// scalastyle:off println
|
||||
package org.apache.spark.examples.sql
|
||||
|
||||
import org.apache.spark.{SparkConf, SparkContext}
|
||||
import org.apache.spark.sql.{SaveMode, SparkSession}
|
||||
|
||||
// One method for defining the schema of an RDD is to make a case class with the desired column
|
||||
|
@ -27,14 +26,12 @@ case class Record(key: Int, value: String)
|
|||
|
||||
object RDDRelation {
|
||||
def main(args: Array[String]) {
|
||||
val sparkConf = new SparkConf().setAppName("RDDRelation")
|
||||
val sc = new SparkContext(sparkConf)
|
||||
val spark = new SparkSession(sc)
|
||||
val spark = SparkSession.builder.appName("RDDRelation").getOrCreate()
|
||||
|
||||
// Importing the SparkSession gives access to all the SQL functions and implicit conversions.
|
||||
import spark.implicits._
|
||||
|
||||
val df = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))).toDF()
|
||||
val df = spark.createDataFrame((1 to 100).map(i => Record(i, s"val_$i")))
|
||||
// Any RDD containing case classes can be registered as a table. The schema of the table is
|
||||
// automatically inferred using scala reflection.
|
||||
df.registerTempTable("records")
|
||||
|
@ -70,7 +67,7 @@ object RDDRelation {
|
|||
parquetFile.registerTempTable("parquetFile")
|
||||
spark.sql("SELECT * FROM parquetFile").collect().foreach(println)
|
||||
|
||||
sc.stop()
|
||||
spark.stop()
|
||||
}
|
||||
}
|
||||
// scalastyle:on println
|
||||
|
|
|
@ -1030,10 +1030,10 @@ class SparkILoop(
|
|||
def createSparkSession(): SparkSession = {
|
||||
if (SparkSession.hiveClassesArePresent) {
|
||||
logInfo("Creating Spark session with Hive support")
|
||||
SparkSession.withHiveSupport(sparkContext)
|
||||
SparkSession.builder.enableHiveSupport().getOrCreate()
|
||||
} else {
|
||||
logInfo("Creating Spark session")
|
||||
new SparkSession(sparkContext)
|
||||
SparkSession.builder.getOrCreate()
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -94,10 +94,10 @@ object Main extends Logging {
|
|||
|
||||
def createSparkSession(): SparkSession = {
|
||||
if (SparkSession.hiveClassesArePresent) {
|
||||
sparkSession = SparkSession.withHiveSupport(sparkContext)
|
||||
sparkSession = SparkSession.builder.enableHiveSupport().getOrCreate()
|
||||
logInfo("Created Spark session with Hive support")
|
||||
} else {
|
||||
sparkSession = new SparkSession(sparkContext)
|
||||
sparkSession = SparkSession.builder.getOrCreate()
|
||||
logInfo("Created Spark session")
|
||||
}
|
||||
sparkSession
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
Loading…
Reference in a new issue