Revert "[SPARK-20946][SQL] simplify the config setting logic in SparkSession.getOrCreate"

This reverts commit e11d90bf8d.
This commit is contained in:
Yin Huai 2017-06-02 15:36:21 -07:00
parent 2a780ac7fe
commit 0eb1fc6cd5
3 changed files with 21 additions and 10 deletions

View file

@ -820,13 +820,15 @@ class ALSCleanerSuite extends SparkFunSuite {
FileUtils.listFiles(localDir, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).asScala.toSet
try {
conf.set("spark.local.dir", localDir.getAbsolutePath)
val sc = new SparkContext("local[2]", "ALSCleanerSuite", conf)
val sc = new SparkContext("local[2]", "test", conf)
try {
sc.setCheckpointDir(checkpointDir.getAbsolutePath)
// Generate test data
val (training, _) = ALSSuite.genImplicitTestData(sc, 20, 5, 1, 0.2, 0)
// Implicitly test the cleaning of parents during ALS training
val spark = SparkSession.builder
.master("local[2]")
.appName("ALSCleanerSuite")
.sparkContext(sc)
.getOrCreate()
import spark.implicits._

View file

@ -43,6 +43,8 @@ private[ml] object TreeTests extends SparkFunSuite {
categoricalFeatures: Map[Int, Int],
numClasses: Int): DataFrame = {
val spark = SparkSession.builder()
.master("local[2]")
.appName("TreeTests")
.sparkContext(data.sparkContext)
.getOrCreate()
import spark.implicits._

View file

@ -757,8 +757,6 @@ object SparkSession {
private[this] var userSuppliedContext: Option[SparkContext] = None
// The `SparkConf` inside the given `SparkContext` may get changed if you specify some options
// for this builder.
private[spark] def sparkContext(sparkContext: SparkContext): Builder = synchronized {
userSuppliedContext = Option(sparkContext)
this
@ -856,7 +854,7 @@ object SparkSession {
*
* @since 2.2.0
*/
def withExtensions(f: SparkSessionExtensions => Unit): Builder = synchronized {
def withExtensions(f: SparkSessionExtensions => Unit): Builder = {
f(extensions)
this
}
@ -901,14 +899,22 @@ object SparkSession {
// No active nor global default session. Create a new one.
val sparkContext = userSuppliedContext.getOrElse {
// set app name if not given
val randomAppName = java.util.UUID.randomUUID().toString
val sparkConf = new SparkConf()
options.get("spark.master").foreach(sparkConf.setMaster)
// set a random app name if not given.
sparkConf.setAppName(options.getOrElse("spark.app.name",
java.util.UUID.randomUUID().toString))
SparkContext.getOrCreate(sparkConf)
options.foreach { case (k, v) => sparkConf.set(k, v) }
if (!sparkConf.contains("spark.app.name")) {
sparkConf.setAppName(randomAppName)
}
val sc = SparkContext.getOrCreate(sparkConf)
// maybe this is an existing SparkContext, update its SparkConf which maybe used
// by SparkSession
options.foreach { case (k, v) => sc.conf.set(k, v) }
if (!sc.conf.contains("spark.app.name")) {
sc.conf.setAppName(randomAppName)
}
sc
}
options.foreach { case (k, v) => sparkContext.conf.set(k, v) }
// Initialize extensions if the user has defined a configurator class.
val extensionConfOption = sparkContext.conf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS)
@ -929,6 +935,7 @@ object SparkSession {
}
session = new SparkSession(sparkContext, None, None, extensions)
options.foreach { case (k, v) => session.sessionState.conf.setConfString(k, v) }
defaultSession.set(session)
// Register a successfully instantiated context to the singleton. This should be at the