diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
index adbcdd302aba9ef289c62d7c9b564a63ef9f3280..4bba2ea057af72ee7d063ad37d55c11b57b9d253 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
@@ -437,7 +437,7 @@ class LogisticRegressionWithLBFGS
         lr.setMaxIter(optimizer.getNumIterations())
         lr.setTol(optimizer.getConvergenceTol())
         // Convert our input into a DataFrame
-        val sqlContext = new SQLContext(input.context)
+        val sqlContext = SQLContext.getOrCreate(input.context)
         import sqlContext.implicits._
         val df = input.map(_.asML).toDF()
         // Determine if we should cache the DF
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index 08c575aaeec784335c82e7ffaaefb1482f651c50..73debe9da427732ba77bd7f40a037660dde1abb0 100644
--- a/project/MimaExcludes.scala
+++ b/project/MimaExcludes.scala
@@ -54,6 +54,8 @@ object MimaExcludes {
         ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.rdd.RDD.coalesce"),
         ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.rdd.PartitionCoalescer$LocationIterator"),
         ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.rdd.PartitionCoalescer"),
+        // SPARK-15532 Remove isRootContext flag from SQLContext.
+        ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.isRootContext"),
         // SPARK-12600 Remove SQL deprecated methods
         ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$QueryExecution"),
         ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$SparkPlanner"),
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index b17fb8a839487a81cadcd083a99a10667b903b5d..66d9aa2c857991dccb667ec5a3b7f17b5dde4c8e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -57,9 +57,7 @@ import org.apache.spark.sql.util.ExecutionListenerManager
  * @groupname Ungrouped Support functions for language integrated queries
  * @since 1.0.0
  */
-class SQLContext private[sql](
-    val sparkSession: SparkSession,
-    val isRootContext: Boolean)
+class SQLContext private[sql](val sparkSession: SparkSession)
   extends Logging with Serializable {
 
   self =>
@@ -69,13 +67,9 @@ class SQLContext private[sql](
   // Note: Since Spark 2.0 this class has become a wrapper of SparkSession, where the
   // real functionality resides. This class remains mainly for backward compatibility.
 
-  private[sql] def this(sparkSession: SparkSession) = {
-    this(sparkSession, true)
-  }
-
   @deprecated("Use SparkSession.builder instead", "2.0.0")
   def this(sc: SparkContext) = {
-    this(new SparkSession(sc))
+    this(SparkSession.builder().sparkContext(sc).getOrCreate())
   }
 
   @deprecated("Use SparkSession.builder instead", "2.0.0")
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 a36368afe22a7e2a2a806f0aff7983df666cbb0c..5dabe0e83c1cfd80d0a32ea6886ef374b154d7f2 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
@@ -25,7 +25,7 @@ import scala.reflect.ClassTag
 import scala.reflect.runtime.universe.TypeTag
 import scala.util.control.NonFatal
 
-import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.{SparkConf, SparkContext, SparkException}
 import org.apache.spark.annotation.{DeveloperApi, Experimental}
 import org.apache.spark.api.java.JavaRDD
 import org.apache.spark.internal.Logging
@@ -40,7 +40,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Range}
 import org.apache.spark.sql.execution._
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.execution.ui.SQLListener
-import org.apache.spark.sql.internal.{CatalogImpl, SessionState, SharedState}
+import org.apache.spark.sql.internal.{CatalogImpl, SessionState, SharedState, SQLConf}
 import org.apache.spark.sql.sources.BaseRelation
 import org.apache.spark.sql.types.{DataType, LongType, StructType}
 import org.apache.spark.sql.util.ExecutionListenerManager
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index 415d4c0049d4076a32f75eaff51c1bdd6af2b07a..3cfe93234f24bfd3349a27f041c4ddc9b8fc5742 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -28,15 +28,13 @@ import org.apache.spark.sql.{SparkSession, SQLContext}
  * Configuration for Hive is read from hive-site.xml on the classpath.
  */
 @deprecated("Use SparkSession.builder.enableHiveSupport instead", "2.0.0")
-class HiveContext private[hive](
-    _sparkSession: SparkSession,
-    isRootContext: Boolean)
-  extends SQLContext(_sparkSession, isRootContext) with Logging {
+class HiveContext private[hive](_sparkSession: SparkSession)
+  extends SQLContext(_sparkSession) with Logging {
 
   self =>
 
   def this(sc: SparkContext) = {
-    this(new SparkSession(HiveUtils.withHiveExternalCatalog(sc)), true)
+    this(SparkSession.builder().sparkContext(HiveUtils.withHiveExternalCatalog(sc)).getOrCreate())
   }
 
   def this(sc: JavaSparkContext) = this(sc.sc)
@@ -47,7 +45,7 @@ class HiveContext private[hive](
    * and Hive client (both of execution and metadata) with existing HiveContext.
    */
   override def newSession(): HiveContext = {
-    new HiveContext(sparkSession.newSession(), isRootContext = false)
+    new HiveContext(sparkSession.newSession())
   }
 
   protected[sql] override def sessionState: HiveSessionState = {
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
index 2f6aa36f95827f76af377566d17df850b1cfee6d..81964db5477c667f23bf48215bc87408799ab8fe 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
@@ -70,16 +70,15 @@ object TestHive
  * test cases that rely on TestHive must be serialized.
  */
 class TestHiveContext(
-    @transient override val sparkSession: TestHiveSparkSession,
-    isRootContext: Boolean)
-  extends SQLContext(sparkSession, isRootContext) {
+    @transient override val sparkSession: TestHiveSparkSession)
+  extends SQLContext(sparkSession) {
 
   def this(sc: SparkContext) {
-    this(new TestHiveSparkSession(HiveUtils.withHiveExternalCatalog(sc)), true)
+    this(new TestHiveSparkSession(HiveUtils.withHiveExternalCatalog(sc)))
   }
 
   override def newSession(): TestHiveContext = {
-    new TestHiveContext(sparkSession.newSession(), false)
+    new TestHiveContext(sparkSession.newSession())
   }
 
   override def sharedState: TestHiveSharedState = sparkSession.sharedState