Skip to content
Prev Previous commit
Next Next commit
add more ut
  • Loading branch information
yaooqinn committed Feb 1, 2019
commit 0cac5e6c353c32e71ae2bd49888d4755918acd7b
Original file line number Diff line number Diff line change
Expand Up @@ -17,28 +17,54 @@

package org.apache.spark.sql.hive

import org.apache.hadoop.hive.conf.HiveConf.ConfVars

import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.internal.SharedState
import org.apache.spark.sql.internal.StaticSQLConf._
import org.apache.spark.util.Utils

class HiveSharedStateSuite extends SparkFunSuite {

test("the catalog should be determined at the very first") {
SparkContext.getActive.foreach(_.stop())
var sc: SparkContext = null
try {
val conf = new SparkConf().setMaster("local").setAppName("SharedState Test")
sc = new SparkContext(conf)
val ss = SparkSession.builder().enableHiveSupport().getOrCreate()
assert(ss.sharedState.externalCatalog.unwrapped.getClass.getName
.contains("HiveExternalCatalog"), "The catalog should be hive ")

val ss2 = SparkSession.builder().getOrCreate()
assert(ss2.sharedState.externalCatalog.unwrapped.getClass.getName
.contains("HiveExternalCatalog"), "The catalog should be shared across sessions")
} finally {
if (sc != null && !sc.isStopped) {
sc.stop()
}
}
val conf = new SparkConf().setMaster("local").setAppName("SharedState Test")
val sc = SparkContext.getOrCreate(conf)
val ss = SparkSession.builder().enableHiveSupport().getOrCreate()
assert(ss.sharedState.externalCatalog.unwrapped.getClass.getName
.contains("HiveExternalCatalog"), "The catalog should be hive ")

val ss2 = SparkSession.builder().getOrCreate()
assert(ss2.sharedState.externalCatalog.unwrapped.getClass.getName
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: use isInstanceOf[HiveExternalCatalog]

.contains("HiveExternalCatalog"), "The catalog should be shared across sessions")

}

test("using initial configs to generate SharedState") {
Copy link
Contributor

Choose a reason for hiding this comment

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

IIUC these 2 tests are positive and negative cases for setting initial session configs. Can you make it more explicit in the test name, and move all the configs that can take effect during session creation to one test?

val conf = new SparkConf().setMaster("local").setAppName("SharedState Test")
val sc = SparkContext.getOrCreate(conf)
val invalidPath = "invalid/path"
val metastorePath = Utils.createTempDir()
val tmpDb = "tmp_db"
val initialConfigs = Map("spark.foo" -> "bar",
WAREHOUSE_PATH.key -> invalidPath,
ConfVars.METASTOREWAREHOUSE.varname -> invalidPath,
CATALOG_IMPLEMENTATION.key -> "hive",
ConfVars.METASTORECONNECTURLKEY.varname ->
s"jdbc:derby:;databaseName=$metastorePath/metastore_db;create=true",
GLOBAL_TEMP_DATABASE.key -> tmpDb)
val state = new SharedState(sc, initialConfigs)
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we be more real-world and create SparkSession here?

Copy link
Member Author

Choose a reason for hiding this comment

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

In hive module test, there is an long cached TestHiveSession, so I use this SharedState explicitly

Copy link
Contributor

Choose a reason for hiding this comment

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

why can we build SparkSession in the above test case?

Copy link
Member Author

Choose a reason for hiding this comment

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

Individually the above test runs correctly, but will use a existing SparkSession with hive support while runs under the whole hive module. so I add the second test to insure all configurations passed correctly.

Copy link
Contributor

Choose a reason for hiding this comment

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

shall we add beforeEach and call SparkSession.clearActiveSession and SparkSession.clearDefaultSession?

Copy link
Member Author

Choose a reason for hiding this comment

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

without this fix ,test 1 fails individually but pass fully

Copy link
Contributor

Choose a reason for hiding this comment

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

I'd like to move test 1 to sql core module, we can catch the IllegalArgumentException to check it. Then the test is reliable, it doesn't matter we run it individually or with other tests.

Copy link
Member Author

Choose a reason for hiding this comment

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

IllegalArgumentException does not mean that the catalog is hive or not. It will also pass anyway with or without this fix,and the fact is that it doesn't have the chance to get to where the catalog is initialized.

Copy link
Contributor

Choose a reason for hiding this comment

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

then let's just remove test 1, but post the code in PR description to let people know how to reproduce the bug.

Copy link
Member Author

Choose a reason for hiding this comment

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

ok. both done.

assert(state.warehousePath !== invalidPath, "warehouse path can't determine by session options")
assert(sc.conf.get(WAREHOUSE_PATH.key) !== invalidPath,
"warehouse conf in session options can't affect application wide spark conf")
assert(sc.hadoopConfiguration.get(ConfVars.METASTOREWAREHOUSE.varname) !== invalidPath,
"warehouse conf in session options can't affect application wide hadoop conf")

assert(!state.sparkContext.conf.contains("spark.foo"),
"static spark conf should not be affected by session")
assert(state.sparkContext.conf.get(CATALOG_IMPLEMENTATION) === "in-memory")
assert(state.globalTempViewManager.database === tmpDb)
assert(state.externalCatalog.unwrapped.isInstanceOf[HiveExternalCatalog],
"Initial SparkSession options can determine the catalog")
}
}