diff --git a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala
index f69d46cd17d0b2713b84678f34d1fd9591bc0f9f..8cbfb9cd41b389ad6dde1f4a06a643e19153066b 100644
--- a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala
+++ b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala
@@ -33,7 +33,9 @@ object SparkSqlExample {
       case None => new SparkConf().setAppName("Simple Sql App")
     }
     val sc = new SparkContext(conf)
-    val sparkSession = SparkSession.withHiveSupport(sc)
+    val sparkSession = SparkSession.builder
+      .enableHiveSupport()
+      .getOrCreate()
 
     import sparkSession._
     sql("DROP TABLE IF EXISTS src")
@@ -41,14 +43,14 @@ object SparkSqlExample {
     sql("LOAD DATA LOCAL INPATH 'data.txt' INTO TABLE src")
     val results = sql("FROM src SELECT key, value WHERE key >= 0 AND KEY < 5").collect()
     results.foreach(println)
-    
+
     def test(f: => Boolean, failureMsg: String) = {
       if (!f) {
         println(failureMsg)
         System.exit(-1)
       }
     }
-    
+
     test(results.size == 5, "Unexpected number of selected elements: " + results)
     println("Test succeeded")
     sc.stop()
diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py
index c6b0eda9961686c8d4f6d66f504eeeb2d20cb5e4..adaa3b5a799383c34fb402aa1c640bda4bfa0871 100644
--- a/python/pyspark/shell.py
+++ b/python/pyspark/shell.py
@@ -41,7 +41,9 @@ atexit.register(lambda: sc.stop())
 try:
     # Try to access HiveConf, it will raise exception if Hive is not added
     sc._jvm.org.apache.hadoop.hive.conf.HiveConf()
-    spark = SparkSession.withHiveSupport(sc)
+    spark = SparkSession.builder\
+        .enableHiveSupport()\
+        .getOrCreate()
 except py4j.protocol.Py4JError:
     spark = SparkSession(sc)
 except TypeError:
diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py
index 04842f6185c713ffbd3bc0a8b04ecb9a0b2b75f5..4ee9ab8ab2f222aeb5baca63dd04eb7be7583b3f 100644
--- a/python/pyspark/sql/session.py
+++ b/python/pyspark/sql/session.py
@@ -182,16 +182,6 @@ class SparkSession(object):
         if SparkSession._instantiatedContext is None:
             SparkSession._instantiatedContext = self
 
-    @classmethod
-    @since(2.0)
-    def withHiveSupport(cls, sparkContext):
-        """Returns a new SparkSession with a catalog backed by Hive.
-
-        :param sparkContext: The underlying :class:`SparkContext`.
-        """
-        jsparkSession = sparkContext._jvm.SparkSession.withHiveSupport(sparkContext._jsc.sc())
-        return cls(sparkContext, jsparkSession)
-
     @since(2.0)
     def newSession(self):
         """
diff --git a/sql/hivecontext-compatibility/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hivecontext-compatibility/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index aa0485a8912ed9bdcf612e9bfe1a831f91055411..75166f6beaa8798c9e99e3112b796cd1d7e31a23 100644
--- a/sql/hivecontext-compatibility/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hivecontext-compatibility/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -27,7 +27,7 @@ import org.apache.spark.sql.{SparkSession, SQLContext}
  * An instance of the Spark SQL execution engine that integrates with data stored in Hive.
  * Configuration for Hive is read from hive-site.xml on the classpath.
  */
-@deprecated("Use SparkSession.withHiveSupport instead", "2.0.0")
+@deprecated("Use SparkSession.builder.enableHiveSupport instead", "2.0.0")
 class HiveContext private[hive](
     _sparkSession: SparkSession,
     isRootContext: Boolean)