From 06168a65816e7a0b01f031c58b0cffbf7306114d Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 20 May 2016 15:12:06 -0700 Subject: [PATCH 1/6] Add a conf to control if we want to share Hadoop classes for IsolatedClientLoader. --- .../org/apache/spark/sql/hive/HiveUtils.scala | 21 +++++++++++++++++++ .../hive/client/IsolatedClientLoader.scala | 11 +++++----- 2 files changed, 27 insertions(+), 5 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index d033b05d4806e..0e3395614a1f5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -129,6 +129,14 @@ private[spark] object HiveUtils extends Logging { .toSequence .createWithDefault(Nil) + val HIVE_METASTORE_SHARE_HADOOPCLASSES = + SQLConfigBuilder("spark.sql.hive.metastore.shareHadoopClasses") + .doc("When set to false, the classloader used to load Hive metastore client will " + + "explicitly load Hadoop classes instead of using classes loaded by the classloader " + + "that loads Spark.") + .booleanConf + .createWithDefault(true) + val HIVE_THRIFT_SERVER_ASYNC = SQLConfigBuilder("spark.sql.hive.thriftServer.async") .doc("When set to true, Hive Thrift server executes SQL queries in an asynchronous way.") .booleanConf @@ -175,6 +183,15 @@ private[spark] object HiveUtils extends Logging { conf.getConf(HIVE_METASTORE_BARRIER_PREFIXES).filterNot(_ == "") } + /** + * A comma separated list of class prefixes that should explicitly be reloaded for each version + * of Hive that Spark SQL is communicating with. For example, Hive UDFs that are declared in a + * prefix that typically would be shared (i.e. org.apache.spark.*) + */ + private def hiveMetastoreShareHadoopClasses(conf: SQLConf): Boolean = { + conf.getConf(HIVE_METASTORE_SHARE_HADOOPCLASSES) + } + /** * Configurations needed to create a [[HiveClient]]. */ @@ -278,6 +295,7 @@ private[spark] object HiveUtils extends Logging { val hiveMetastoreJars = HiveUtils.hiveMetastoreJars(sqlConf) val hiveMetastoreSharedPrefixes = HiveUtils.hiveMetastoreSharedPrefixes(sqlConf) val hiveMetastoreBarrierPrefixes = HiveUtils.hiveMetastoreBarrierPrefixes(sqlConf) + val hiveMetastoreShareHadoopClasses = HiveUtils.hiveMetastoreShareHadoopClasses(sqlConf) val metaVersion = IsolatedClientLoader.hiveVersion(hiveMetastoreVersion) val isolatedLoader = if (hiveMetastoreJars == "builtin") { @@ -315,6 +333,7 @@ private[spark] object HiveUtils extends Logging { execJars = jars.toSeq, config = configurations, isolationOn = true, + sharesHadoopClasses = hiveMetastoreShareHadoopClasses, barrierPrefixes = hiveMetastoreBarrierPrefixes, sharedPrefixes = hiveMetastoreSharedPrefixes) } else if (hiveMetastoreJars == "maven") { @@ -326,6 +345,7 @@ private[spark] object HiveUtils extends Logging { hadoopVersion = VersionInfo.getVersion, sparkConf = conf, hadoopConf = hadoopConf, + sharesHadoopClasses = hiveMetastoreShareHadoopClasses, config = configurations, barrierPrefixes = hiveMetastoreBarrierPrefixes, sharedPrefixes = hiveMetastoreSharedPrefixes) @@ -358,6 +378,7 @@ private[spark] object HiveUtils extends Logging { execJars = jars.toSeq, config = configurations, isolationOn = true, + sharesHadoopClasses = hiveMetastoreShareHadoopClasses, barrierPrefixes = hiveMetastoreBarrierPrefixes, sharedPrefixes = hiveMetastoreSharedPrefixes) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index e1950d181d10e..65dda7636d100 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -46,17 +46,18 @@ private[hive] object IsolatedClientLoader extends Logging { hadoopVersion: String, sparkConf: SparkConf, hadoopConf: Configuration, + sharesHadoopClasses: Boolean = true, config: Map[String, String] = Map.empty, ivyPath: Option[String] = None, sharedPrefixes: Seq[String] = Seq.empty, barrierPrefixes: Seq[String] = Seq.empty): IsolatedClientLoader = synchronized { val resolvedVersion = hiveVersion(hiveMetastoreVersion) - // We will first try to share Hadoop classes. If we cannot resolve the Hadoop artifact - // with the given version, we will use Hadoop 2.4.0 and then will not share Hadoop classes. - var sharesHadoopClasses = true + var newSharesHadoopClasses = sharesHadoopClasses val files = if (resolvedVersions.contains((resolvedVersion, hadoopVersion))) { resolvedVersions((resolvedVersion, hadoopVersion)) } else { + // If we cannot resolve the Hadoop artifact with the given version, we will use Hadoop 2.4.0 + // and then will not share Hadoop classes. val (downloadedFiles, actualHadoopVersion) = try { (downloadVersion(resolvedVersion, hadoopVersion, ivyPath), hadoopVersion) @@ -72,7 +73,7 @@ private[hive] object IsolatedClientLoader extends Logging { "Hadoop classes will not be shared between Spark and Hive metastore client. " + "It is recommended to set jars used by Hive metastore client through " + "spark.sql.hive.metastore.jars in the production environment.") - sharesHadoopClasses = false + newSharesHadoopClasses = false (downloadVersion(resolvedVersion, "2.4.0", ivyPath), "2.4.0") } resolvedVersions.put((resolvedVersion, actualHadoopVersion), downloadedFiles) @@ -85,7 +86,7 @@ private[hive] object IsolatedClientLoader extends Logging { execJars = files, hadoopConf = hadoopConf, config = config, - sharesHadoopClasses = sharesHadoopClasses, + sharesHadoopClasses = newSharesHadoopClasses, sharedPrefixes = sharedPrefixes, barrierPrefixes = barrierPrefixes) } From 4639c8583f8ff7b45c0b4ceb48df2aefa0288691 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 20 May 2016 15:54:13 -0700 Subject: [PATCH 2/6] Do not pass hadoop Configuration to HiveClientImpl --- .../sql/hive/client/HiveClientImpl.scala | 44 +++++++++---------- .../hive/client/IsolatedClientLoader.scala | 6 ++- 2 files changed, 26 insertions(+), 24 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index af2850d4f568c..657a4f0f7b721 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -63,8 +63,9 @@ import org.apache.spark.util.{CircularBuffer, Utils} * * @param version the version of hive used when pick function calls that are not compatible. * @param sparkConf all configuration options set in SparkConf. - * @param hadoopConf the base Configuration object used by the HiveConf created inside - * this HiveClientImpl. + * @param hadoopConf the map containing all entries of the base Hadoop Configuration. + * We cannot pass in Hadoop Configuration to here because Hadoop classes + * can be loaded by the IsolatedClientLoader. * @param extraConfig a collection of configuration options that will be added to the * hive conf before opening the hive client. * @param initClassLoader the classloader used when creating the `state` field of @@ -73,7 +74,7 @@ import org.apache.spark.util.{CircularBuffer, Utils} private[hive] class HiveClientImpl( override val version: HiveVersion, sparkConf: SparkConf, - hadoopConf: Configuration, + hadoopConf: Map[String, String], extraConfig: Map[String, String], initClassLoader: ClassLoader, val clientLoader: IsolatedClientLoader) @@ -127,6 +128,17 @@ private[hive] class HiveClientImpl( found } + def setToHiveConf(hiveConf: HiveConf, confs: Map[String, String]): Unit = { + confs.foreach { case (k, v) => + if (k.toLowerCase.contains("password")) { + logDebug(s"Applying Spark config to Hive Conf: $k=xxx") + } else { + logDebug(s"Applying Spark config to Hive Conf: $k=$v") + } + hiveConf.set(k, v) + } + } + val ret = try { // originState will be created if not exists, will never be null val originalState = SessionState.get() @@ -137,31 +149,19 @@ private[hive] class HiveClientImpl( // so we should keep `conf` and reuse the existing instance of `CliSessionState`. originalState } else { - val hiveConf = new HiveConf(hadoopConf, classOf[SessionState]) + val hiveConf = new HiveConf(classOf[SessionState]) // HiveConf is a Hadoop Configuration, which has a field of classLoader and // the initial value will be the current thread's context class loader // (i.e. initClassLoader at here). // We call initialConf.setClassLoader(initClassLoader) at here to make // this action explicit. hiveConf.setClassLoader(initClassLoader) - // First, we set all spark confs to this hiveConf. - sparkConf.getAll.foreach { case (k, v) => - if (k.toLowerCase.contains("password")) { - logDebug(s"Applying Spark config to Hive Conf: $k=xxx") - } else { - logDebug(s"Applying Spark config to Hive Conf: $k=$v") - } - hiveConf.set(k, v) - } - // Second, we set all entries in config to this hiveConf. - extraConfig.foreach { case (k, v) => - if (k.toLowerCase.contains("password")) { - logDebug(s"Applying extra config to HiveConf: $k=xxx") - } else { - logDebug(s"Applying extra config to HiveConf: $k=$v") - } - hiveConf.set(k, v) - } + // First, we set all Hadoop confs to this hiveConf. + setToHiveConf(hiveConf, hadoopConf) + // Second, we set all spark confs to this hiveConf. + setToHiveConf(hiveConf, sparkConf.getAll.toMap) + // Third, we set all entries in config to this hiveConf. + setToHiveConf(hiveConf, extraConfig) val state = new SessionState(hiveConf) if (clientLoader.cachedHive != null) { Hive.set(clientLoader.cachedHive.asInstanceOf[Hive]) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 65dda7636d100..72094e165fc1c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -22,6 +22,7 @@ import java.lang.reflect.InvocationTargetException import java.net.{URL, URLClassLoader} import java.util +import scala.collection.JavaConverters._ import scala.language.reflectiveCalls import scala.util.Try @@ -244,8 +245,9 @@ private[hive] class IsolatedClientLoader( /** The isolated client interface to Hive. */ private[hive] def createClient(): HiveClient = { + val hadoopConfMap = hadoopConf.asScala.map(e => e.getKey -> e.getValue).toMap if (!isolationOn) { - return new HiveClientImpl(version, sparkConf, hadoopConf, config, baseClassLoader, this) + return new HiveClientImpl(version, sparkConf, hadoopConfMap, config, baseClassLoader, this) } // Pre-reflective instantiation setup. logDebug("Initializing the logger to avoid disaster...") @@ -256,7 +258,7 @@ private[hive] class IsolatedClientLoader( classLoader .loadClass(classOf[HiveClientImpl].getName) .getConstructors.head - .newInstance(version, sparkConf, hadoopConf, config, classLoader, this) + .newInstance(version, sparkConf, hadoopConfMap, config, classLoader, this) .asInstanceOf[HiveClient] } catch { case e: InvocationTargetException => From c0e3ae027e72e3baab687cb7a4e97ff39ca09592 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 20 May 2016 16:01:03 -0700 Subject: [PATCH 3/6] style --- .../main/scala/org/apache/spark/sql/hive/HiveUtils.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index 0e3395614a1f5..e80ce7cedd93d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -184,10 +184,10 @@ private[spark] object HiveUtils extends Logging { } /** - * A comma separated list of class prefixes that should explicitly be reloaded for each version - * of Hive that Spark SQL is communicating with. For example, Hive UDFs that are declared in a - * prefix that typically would be shared (i.e. org.apache.spark.*) - */ + * A comma separated list of class prefixes that should explicitly be reloaded for each version + * of Hive that Spark SQL is communicating with. For example, Hive UDFs that are declared in a + * prefix that typically would be shared (i.e. org.apache.spark.*) + */ private def hiveMetastoreShareHadoopClasses(conf: SQLConf): Boolean = { conf.getConf(HIVE_METASTORE_SHARE_HADOOPCLASSES) } From c69626487593c2ec179e92d0cf4baa4a3d4e6e92 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 20 May 2016 17:01:38 -0700 Subject: [PATCH 4/6] update --- .../src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala | 2 +- .../org/apache/spark/sql/hive/client/IsolatedClientLoader.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index e80ce7cedd93d..b561ee24e92c7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -344,8 +344,8 @@ private[spark] object HiveUtils extends Logging { hiveMetastoreVersion = hiveMetastoreVersion, hadoopVersion = VersionInfo.getVersion, sparkConf = conf, - hadoopConf = hadoopConf, sharesHadoopClasses = hiveMetastoreShareHadoopClasses, + hadoopConf = hadoopConf, config = configurations, barrierPrefixes = hiveMetastoreBarrierPrefixes, sharedPrefixes = hiveMetastoreSharedPrefixes) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 72094e165fc1c..005686d37fbf3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -46,8 +46,8 @@ private[hive] object IsolatedClientLoader extends Logging { hiveMetastoreVersion: String, hadoopVersion: String, sparkConf: SparkConf, - hadoopConf: Configuration, sharesHadoopClasses: Boolean = true, + hadoopConf: Configuration, config: Map[String, String] = Map.empty, ivyPath: Option[String] = None, sharedPrefixes: Seq[String] = Seq.empty, From 1219637650f9eef6ac7877fc7e9366a7b5785534 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 20 May 2016 17:22:14 -0700 Subject: [PATCH 5/6] update doc and default value --- .../main/scala/org/apache/spark/sql/hive/HiveUtils.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index b561ee24e92c7..a4c6fae8bd462 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -131,11 +131,11 @@ private[spark] object HiveUtils extends Logging { val HIVE_METASTORE_SHARE_HADOOPCLASSES = SQLConfigBuilder("spark.sql.hive.metastore.shareHadoopClasses") - .doc("When set to false, the classloader used to load Hive metastore client will " + - "explicitly load Hadoop classes instead of using classes loaded by the classloader " + - "that loads Spark.") + .doc("When set to true, the classloader used to load Hive metastore client will " + + "not explicitly load Hadoop classes. Instead, Hadoop classes loaded by the classloader " + + "that loads Spark will be used.") .booleanConf - .createWithDefault(true) + .createWithDefault(false) val HIVE_THRIFT_SERVER_ASYNC = SQLConfigBuilder("spark.sql.hive.thriftServer.async") .doc("When set to true, Hive Thrift server executes SQL queries in an asynchronous way.") From 084ae00b981c44d4bf2650b3a41745af065a7d02 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 22 May 2016 15:12:50 -0700 Subject: [PATCH 6/6] Use a map to store Hadoop confs instead of using a Hadoop Configuration object. --- .../scala/org/apache/spark/sql/hive/HiveUtils.scala | 9 +++++---- .../spark/sql/hive/client/IsolatedClientLoader.scala | 10 ++++------ .../apache/spark/sql/hive/client/VersionsSuite.scala | 10 ++++++---- 3 files changed, 15 insertions(+), 14 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index a4c6fae8bd462..2d602f5b4b168 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -265,7 +265,7 @@ private[spark] object HiveUtils extends Logging { version = IsolatedClientLoader.hiveVersion(hiveExecutionVersion), sparkConf = conf, execJars = Seq(), - hadoopConf = hadoopConf, + hadoopConf = hadoopConf.asScala.map(e => e.getKey -> e.getValue).toMap, config = newTemporaryConfiguration(useInMemoryDerby = true), isolationOn = false, baseClassLoader = Utils.getContextOrSparkClassLoader) @@ -297,6 +297,7 @@ private[spark] object HiveUtils extends Logging { val hiveMetastoreBarrierPrefixes = HiveUtils.hiveMetastoreBarrierPrefixes(sqlConf) val hiveMetastoreShareHadoopClasses = HiveUtils.hiveMetastoreShareHadoopClasses(sqlConf) val metaVersion = IsolatedClientLoader.hiveVersion(hiveMetastoreVersion) + val hadoopConfMap = hadoopConf.asScala.map(e => e.getKey -> e.getValue).toMap val isolatedLoader = if (hiveMetastoreJars == "builtin") { if (hiveExecutionVersion != hiveMetastoreVersion) { @@ -329,7 +330,7 @@ private[spark] object HiveUtils extends Logging { new IsolatedClientLoader( version = metaVersion, sparkConf = conf, - hadoopConf = hadoopConf, + hadoopConf = hadoopConfMap, execJars = jars.toSeq, config = configurations, isolationOn = true, @@ -345,7 +346,7 @@ private[spark] object HiveUtils extends Logging { hadoopVersion = VersionInfo.getVersion, sparkConf = conf, sharesHadoopClasses = hiveMetastoreShareHadoopClasses, - hadoopConf = hadoopConf, + hadoopConf = hadoopConfMap, config = configurations, barrierPrefixes = hiveMetastoreBarrierPrefixes, sharedPrefixes = hiveMetastoreSharedPrefixes) @@ -374,7 +375,7 @@ private[spark] object HiveUtils extends Logging { new IsolatedClientLoader( version = metaVersion, sparkConf = conf, - hadoopConf = hadoopConf, + hadoopConf = hadoopConfMap, execJars = jars.toSeq, config = configurations, isolationOn = true, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 005686d37fbf3..e762b1829b60f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -27,7 +27,6 @@ import scala.language.reflectiveCalls import scala.util.Try import org.apache.commons.io.{FileUtils, IOUtils} -import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkSubmitUtils @@ -47,7 +46,7 @@ private[hive] object IsolatedClientLoader extends Logging { hadoopVersion: String, sparkConf: SparkConf, sharesHadoopClasses: Boolean = true, - hadoopConf: Configuration, + hadoopConf: Map[String, String], config: Map[String, String] = Map.empty, ivyPath: Option[String] = None, sharedPrefixes: Seq[String] = Seq.empty, @@ -156,7 +155,7 @@ private[hive] object IsolatedClientLoader extends Logging { private[hive] class IsolatedClientLoader( val version: HiveVersion, val sparkConf: SparkConf, - val hadoopConf: Configuration, + val hadoopConf: Map[String, String], val execJars: Seq[URL] = Seq.empty, val config: Map[String, String] = Map.empty, val isolationOn: Boolean = true, @@ -245,9 +244,8 @@ private[hive] class IsolatedClientLoader( /** The isolated client interface to Hive. */ private[hive] def createClient(): HiveClient = { - val hadoopConfMap = hadoopConf.asScala.map(e => e.getKey -> e.getValue).toMap if (!isolationOn) { - return new HiveClientImpl(version, sparkConf, hadoopConfMap, config, baseClassLoader, this) + return new HiveClientImpl(version, sparkConf, hadoopConf, config, baseClassLoader, this) } // Pre-reflective instantiation setup. logDebug("Initializing the logger to avoid disaster...") @@ -258,7 +256,7 @@ private[hive] class IsolatedClientLoader( classLoader .loadClass(classOf[HiveClientImpl].getName) .getConstructors.head - .newInstance(version, sparkConf, hadoopConfMap, config, classLoader, this) + .newInstance(version, sparkConf, hadoopConf, config, classLoader, this) .asInstanceOf[HiveClient] } catch { case e: InvocationTargetException => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index d46c4e7b2b50b..6afc454add007 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.hive.client import java.io.{ByteArrayOutputStream, File, PrintStream} +import scala.collection.JavaConverters._ + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -71,7 +73,7 @@ class VersionsSuite extends SparkFunSuite with Logging { hiveMetastoreVersion = HiveUtils.hiveExecutionVersion, hadoopVersion = VersionInfo.getVersion, sparkConf = sparkConf, - hadoopConf = new Configuration(), + hadoopConf = Map.empty[String, String], config = buildConf(), ivyPath = ivyPath).createClient() val db = new CatalogDatabase("default", "desc", "loc", Map()) @@ -85,7 +87,7 @@ class VersionsSuite extends SparkFunSuite with Logging { hiveMetastoreVersion = HiveUtils.hiveExecutionVersion, hadoopVersion = VersionInfo.getVersion, sparkConf = sparkConf, - hadoopConf = hadoopConf, + hadoopConf = hadoopConf.asScala.map(e => e.getKey -> e.getValue).toMap, config = buildConf(), ivyPath = ivyPath).createClient() assert("success" === client.getConf("test", null)) @@ -114,7 +116,7 @@ class VersionsSuite extends SparkFunSuite with Logging { hiveMetastoreVersion = "13", hadoopVersion = VersionInfo.getVersion, sparkConf = sparkConf, - hadoopConf = new Configuration(), + hadoopConf = Map.empty[String, String], config = buildConf(), ivyPath = ivyPath).createClient() } @@ -137,7 +139,7 @@ class VersionsSuite extends SparkFunSuite with Logging { hiveMetastoreVersion = version, hadoopVersion = VersionInfo.getVersion, sparkConf = sparkConf, - hadoopConf = hadoopConf, + hadoopConf = hadoopConf.asScala.map(e => e.getKey -> e.getValue).toMap, config = buildConf(), ivyPath = ivyPath).createClient() }