diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 4b928e600b35515fd20700104c5187fa71d4b508..03bb2c222503f817b15247a461f38bb9cbfc7d52 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -83,7 +83,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { val cliConf = new HiveConf(classOf[SessionState]) // Override the location of the metastore since this is only used for local execution. - HiveContext.newTemporaryConfiguration().foreach { + HiveContext.newTemporaryConfiguration(useInMemoryDerby = false).foreach { case (key, value) => cliConf.set(key, value) } val sessionState = new CliSessionState(cliConf) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index e83941c2ecf66a584c3f82fe5bd1ed9afaecc4ea..5958777b0d06404fa3e52ee5cbbf550119201c12 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -212,7 +212,7 @@ class HiveContext private[hive]( val loader = new IsolatedClientLoader( version = IsolatedClientLoader.hiveVersion(hiveExecutionVersion), execJars = Seq(), - config = newTemporaryConfiguration(), + config = newTemporaryConfiguration(useInMemoryDerby = true), isolationOn = false, baseClassLoader = Utils.getContextOrSparkClassLoader) loader.createClient().asInstanceOf[ClientWrapper] @@ -721,7 +721,9 @@ private[hive] object HiveContext { doc = "TODO") /** Constructs a configuration for hive, where the metastore is located in a temp directory. */ - def newTemporaryConfiguration(): Map[String, String] = { + def newTemporaryConfiguration(useInMemoryDerby: Boolean): Map[String, String] = { + val withInMemoryMode = if (useInMemoryDerby) "memory:" else "" + val tempDir = Utils.createTempDir() val localMetastore = new File(tempDir, "metastore") val propMap: HashMap[String, String] = HashMap() @@ -735,7 +737,7 @@ private[hive] object HiveContext { } propMap.put(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, localMetastore.toURI.toString) propMap.put(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, - s"jdbc:derby:;databaseName=${localMetastore.getAbsolutePath};create=true") + s"jdbc:derby:${withInMemoryMode};databaseName=${localMetastore.getAbsolutePath};create=true") propMap.put("datanucleus.rdbms.datastoreAdapterClassName", "org.datanucleus.store.rdbms.adapter.DerbyAdapter") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 2e2d201bf254db05cc79934e6d2065ea4ff1d86a..97792549bb7ab34d906cc0a4f625d021585727c8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -87,7 +87,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { dir } - private lazy val temporaryConfig = newTemporaryConfiguration() + private lazy val temporaryConfig = newTemporaryConfiguration(useInMemoryDerby = false) /** Sets up the system initially or after a RESET command */ protected override def configure(): Map[String, String] = { 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 92962193311d4fe1a576df58e57364127a0ccd0c..53185fd7751e3daccc1fb93c1ce760f6ae4c958a 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 @@ -80,6 +80,8 @@ class HiveSparkSubmitSuite "--master", "local-cluster[2,1,1024]", "--conf", "spark.ui.enabled=false", "--conf", "spark.master.rest.enabled=false", + "--conf", "spark.sql.hive.metastore.version=0.12", + "--conf", "spark.sql.hive.metastore.jars=maven", "--driver-java-options", "-Dderby.system.durability=test", unusedJar.toString) runSparkSubmit(args)