Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ object HiveFromSpark {
// using HiveQL. Users who do not have an existing Hive deployment can still create a
// HiveContext. When not configured by the hive-site.xml, the context automatically
// creates metastore_db and warehouse in the current directory.
val sparkSession = SparkSession.withHiveSupport(sc)
val sparkSession = SparkSession.builder.enableHiveSupport().getOrCreate()
import sparkSession.implicits._
import sparkSession.sql

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1023,7 +1023,7 @@ class SparkILoop(
val builder = SparkSession.builder.config(conf)
val sparkSession = if (SparkSession.hiveClassesArePresent) {
logInfo("Creating Spark session with Hive support")
builder.enableHiveSupport().getOrCreate()
SparkSession.builder.enableHiveSupport().getOrCreate()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same here

} else {
logInfo("Creating Spark session")
builder.getOrCreate()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@ object Main extends Logging {

val builder = SparkSession.builder.config(conf)
if (SparkSession.hiveClassesArePresent) {
sparkSession = builder.enableHiveSupport().getOrCreate()
sparkSession = SparkSession.builder.enableHiveSupport().getOrCreate()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

wait, this is wrong now. We want to use the builder we already have

logInfo("Created Spark session with Hive support")
} else {
sparkSession = builder.getOrCreate()
Expand Down
13 changes: 0 additions & 13 deletions sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
Original file line number Diff line number Diff line change
Expand Up @@ -816,17 +816,4 @@ object SparkSession {
}
}

/**
* Create a new [[SparkSession]] with a catalog backed by Hive.
*/
def withHiveSupport(sc: SparkContext): SparkSession = {
if (hiveClassesArePresent) {
sc.conf.set(CATALOG_IMPLEMENTATION.key, "hive")
new SparkSession(sc)
} else {
throw new IllegalArgumentException(
"Unable to instantiate SparkSession with Hive support because Hive classes are not found.")
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ private[hive] object SparkSQLEnv extends Logging {
maybeKryoReferenceTracking.getOrElse("false"))

sparkContext = new SparkContext(sparkConf)
sqlContext = SparkSession.withHiveSupport(sparkContext).wrapped
sqlContext = SparkSession.builder.enableHiveSupport().getOrCreate().wrapped
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it would be better if we get the context from the session:

val sparkSession = SparkSession.builder.conf(sparkConf).enableHiveSupport().getOrCreate()
sqlContext = sparkSession.wrapped
sparkContext = sparkSession.sparkContext

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

actually, we should do something like this everywhere else in this patch

val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState]
sessionState.metadataHive.setOut(new PrintStream(System.out, true, "UTF-8"))
sessionState.metadataHive.setInfo(new PrintStream(System.err, true, "UTF-8"))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ object Main {
// scalastyle:off println
println("Running regression test for SPARK-8489.")
val sc = new SparkContext("local", "testing")
val sparkSession = SparkSession.withHiveSupport(sc)
val sparkSession = SparkSession.builder.enableHiveSupport().getOrCreate()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i think this one needs to be built?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not sure I got you ? do you mean something like withSparkContext(sc) instead of getOrCreate()

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In order to use this file, I think it needs to be built into a jar?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yeah we need to rebuild the jar. There's already a TODO on L31. Right now the corresponding test in HiveSparkSubmitSuite is ignored.

Copy link
Contributor Author

@techaddict techaddict May 3, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since we not longer create a assembly jar, not sure how to create this jar easily(steps could be found here #11630).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

let's just fix it later

Copy link
Contributor

@andrewor14 andrewor14 May 5, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this should use the builder's conf method. By the way for this file I would revert the changes here and fix it in #12924 instead, since @dilipbiswal is actually rebuilding the jar there.

// This line should not throw scala.reflect.internal.MissingRequirementError.
// See SPARK-8470 for more detail.
val df = sparkSession.createDataFrame(Seq(MyCoolClass("1", "2", "3")))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -291,7 +291,7 @@ object SetWarehouseLocationTest extends Logging {
conf.set("hive.metastore.warehouse.dir", hiveWarehouseLocation.toString)

val sc = new SparkContext(conf)
val sparkSession = SparkSession.withHiveSupport(sc)
val sparkSession = SparkSession.builder.enableHiveSupport().getOrCreate()
val catalog = sparkSession.sessionState.catalog

sparkSession.sql("drop table if exists testLocation")
Expand Down