Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 5 additions & 3 deletions dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala
Original file line number Diff line number Diff line change
Expand Up @@ -33,22 +33,24 @@ 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")
sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
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()
Expand Down
4 changes: 3 additions & 1 deletion python/pyspark/shell.py
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,9 @@
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:
Expand Down
10 changes: 0 additions & 10 deletions python/pyspark/sql/session.py
Original file line number Diff line number Diff line change
Expand Up @@ -182,16 +182,6 @@ def __init__(self, sparkContext, jsparkSession=None):
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):
"""
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down