aboutsummaryrefslogtreecommitdiff
path: root/sql/hive
diff options
context:
space:
mode:
authortone-zhang <tone.zhang@linaro.org>2016-09-11 10:17:53 +0100
committerSean Owen <sowen@cloudera.com>2016-09-11 10:17:53 +0100
commitbf22217377d7fe95b436d8b563c501aab2797f78 (patch)
tree7de2d7e2110dc6c0ea9d5c42a76747f82b9a1be0 /sql/hive
parent180796ecb3a00facde2d98affdb5aa38dd258875 (diff)
downloadspark-bf22217377d7fe95b436d8b563c501aab2797f78.tar.gz
spark-bf22217377d7fe95b436d8b563c501aab2797f78.tar.bz2
spark-bf22217377d7fe95b436d8b563c501aab2797f78.zip
[SPARK-17330][SPARK UT] Clean up spark-warehouse in UT
## What changes were proposed in this pull request? Check the database warehouse used in Spark UT, and remove the existing database file before run the UT (SPARK-8368). ## How was this patch tested? Run Spark UT with the command for several times: ./build/sbt -Pyarn -Phadoop-2.6 -Phive -Phive-thriftserver "test-only *HiveSparkSubmitSuit*" Without the patch, the test case can be passed only at the first time, and always failed from the second time. With the patch the test case always can be passed correctly. Author: tone-zhang <tone.zhang@linaro.org> Closes #14894 from tone-zhang/issue1.
Diffstat (limited to 'sql/hive')
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala6
1 files changed, 5 insertions, 1 deletions
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
index af28286666..29317e2887 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
@@ -590,7 +590,9 @@ object SparkSubmitClassLoaderTest extends Logging {
def main(args: Array[String]) {
Utils.configTestLog4j("INFO")
val conf = new SparkConf()
+ val hiveWarehouseLocation = Utils.createTempDir()
conf.set("spark.ui.enabled", "false")
+ conf.set("spark.sql.warehouse.dir", hiveWarehouseLocation.toString)
val sc = new SparkContext(conf)
val hiveContext = new TestHiveContext(sc)
val df = hiveContext.createDataFrame((1 to 100).map(i => (i, i))).toDF("i", "j")
@@ -699,11 +701,13 @@ object SPARK_9757 extends QueryTest {
def main(args: Array[String]): Unit = {
Utils.configTestLog4j("INFO")
+ val hiveWarehouseLocation = Utils.createTempDir()
val sparkContext = new SparkContext(
new SparkConf()
.set("spark.sql.hive.metastore.version", "0.13.1")
.set("spark.sql.hive.metastore.jars", "maven")
- .set("spark.ui.enabled", "false"))
+ .set("spark.ui.enabled", "false")
+ .set("spark.sql.warehouse.dir", hiveWarehouseLocation.toString))
val hiveContext = new TestHiveContext(sparkContext)
spark = hiveContext.sparkSession