diff options
author | Marcelo Vanzin <vanzin@cloudera.com> | 2016-03-14 14:27:33 -0700 |
---|---|---|
committer | Marcelo Vanzin <vanzin@cloudera.com> | 2016-03-14 14:27:33 -0700 |
commit | 8301fadd8d269da11e72870b7a889596e3337839 (patch) | |
tree | 74c42c6c99d0438f3669acdae8982f3632259254 /sql | |
parent | 38529d8f2350feb1f143aab0be336050c0f887f2 (diff) | |
download | spark-8301fadd8d269da11e72870b7a889596e3337839.tar.gz spark-8301fadd8d269da11e72870b7a889596e3337839.tar.bz2 spark-8301fadd8d269da11e72870b7a889596e3337839.zip |
[SPARK-13626][CORE] Avoid duplicate config deprecation warnings.
Three different things were needed to get rid of spurious warnings:
- silence deprecation warnings when cloning configuration
- change the way SparkHadoopUtil instantiates SparkConf to silence
warnings
- avoid creating new SparkConf instances where it's not needed.
On top of that, I changed the way that Logging.scala detects the repl;
now it uses a method that is overridden in the repl's Main class, and
the hack in Utils.scala is not needed anymore. This makes the 2.11 repl
behave like the 2.10 one and set the default log level to WARN, which
is a lot better. Previously, this wasn't working because the 2.11 repl
triggers log initialization earlier than the 2.10 one.
I also removed and simplified some other code in the 2.11 repl's Main
to avoid replicating logic that already exists elsewhere in Spark.
Tested the 2.11 repl in local and yarn modes.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes #11510 from vanzin/SPARK-13626.
Diffstat (limited to 'sql')
5 files changed, 21 insertions, 8 deletions
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 9725dcfde1..8244dd4230 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 @@ -210,6 +210,7 @@ class HiveContext private[hive]( logInfo(s"Initializing execution hive, version $hiveExecutionVersion") val loader = new IsolatedClientLoader( version = IsolatedClientLoader.hiveVersion(hiveExecutionVersion), + sparkConf = sc.conf, execJars = Seq(), config = newTemporaryConfiguration(useInMemoryDerby = true), isolationOn = false, @@ -278,6 +279,7 @@ class HiveContext private[hive]( s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using Spark classes.") new IsolatedClientLoader( version = metaVersion, + sparkConf = sc.conf, execJars = jars.toSeq, config = allConfig, isolationOn = true, @@ -290,6 +292,7 @@ class HiveContext private[hive]( IsolatedClientLoader.forVersion( hiveMetastoreVersion = hiveMetastoreVersion, hadoopVersion = VersionInfo.getVersion, + sparkConf = sc.conf, config = allConfig, barrierPrefixes = hiveMetastoreBarrierPrefixes, sharedPrefixes = hiveMetastoreSharedPrefixes) @@ -317,6 +320,7 @@ class HiveContext private[hive]( s"using ${jars.mkString(":")}") new IsolatedClientLoader( version = metaVersion, + sparkConf = sc.conf, execJars = jars.toSeq, config = allConfig, isolationOn = true, 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 c1c8e631ee..c108750c38 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 @@ -60,6 +60,7 @@ import org.apache.spark.util.{CircularBuffer, Utils} */ private[hive] class HiveClientImpl( override val version: HiveVersion, + sparkConf: SparkConf, config: Map[String, String], initClassLoader: ClassLoader, val clientLoader: IsolatedClientLoader) @@ -90,7 +91,6 @@ private[hive] class HiveClientImpl( // instance of SparkConf is needed for the original value of spark.yarn.keytab // and spark.yarn.principal set in SparkSubmit, as yarn.Client resets the // keytab configuration for the link name in distributed cache - val sparkConf = new SparkConf if (sparkConf.contains("spark.yarn.principal") && sparkConf.contains("spark.yarn.keytab")) { val principalName = sparkConf.get("spark.yarn.principal") val keytabFileName = sparkConf.get("spark.yarn.keytab") 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 1653371d89..024f4dfeba 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,7 @@ import scala.util.Try import org.apache.commons.io.{FileUtils, IOUtils} -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.SparkSubmitUtils import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.hive.HiveContext @@ -41,6 +41,7 @@ private[hive] object IsolatedClientLoader extends Logging { def forVersion( hiveMetastoreVersion: String, hadoopVersion: String, + sparkConf: SparkConf, config: Map[String, String] = Map.empty, ivyPath: Option[String] = None, sharedPrefixes: Seq[String] = Seq.empty, @@ -75,7 +76,8 @@ private[hive] object IsolatedClientLoader extends Logging { } new IsolatedClientLoader( - version = hiveVersion(hiveMetastoreVersion), + hiveVersion(hiveMetastoreVersion), + sparkConf, execJars = files, config = config, sharesHadoopClasses = sharesHadoopClasses, @@ -146,6 +148,7 @@ private[hive] object IsolatedClientLoader extends Logging { */ private[hive] class IsolatedClientLoader( val version: HiveVersion, + val sparkConf: SparkConf, val execJars: Seq[URL] = Seq.empty, val config: Map[String, String] = Map.empty, val isolationOn: Boolean = true, @@ -235,7 +238,7 @@ private[hive] class IsolatedClientLoader( /** The isolated client interface to Hive. */ private[hive] def createClient(): HiveClient = { if (!isolationOn) { - return new HiveClientImpl(version, config, baseClassLoader, this) + return new HiveClientImpl(version, sparkConf, config, baseClassLoader, this) } // Pre-reflective instantiation setup. logDebug("Initializing the logger to avoid disaster...") @@ -246,7 +249,7 @@ private[hive] class IsolatedClientLoader( classLoader .loadClass(classOf[HiveClientImpl].getName) .getConstructors.head - .newInstance(version, config, classLoader, this) + .newInstance(version, sparkConf, config, classLoader, this) .asInstanceOf[HiveClient] } catch { case e: InvocationTargetException => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveCatalogSuite.scala index f557abcd52..2809f9439b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveCatalogSuite.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.hive import org.apache.hadoop.util.VersionInfo +import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.hive.client.{HiveClient, IsolatedClientLoader} import org.apache.spark.util.Utils - /** * Test suite for the [[HiveCatalog]]. */ @@ -32,7 +32,8 @@ class HiveCatalogSuite extends CatalogTestCases { private val client: HiveClient = { IsolatedClientLoader.forVersion( hiveMetastoreVersion = HiveContext.hiveExecutionVersion, - hadoopVersion = VersionInfo.getVersion).createClient() + hadoopVersion = VersionInfo.getVersion, + sparkConf = new SparkConf()).createClient() } protected override val tableInputFormat: String = 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 d850d522be..6292f6c3af 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 @@ -21,7 +21,7 @@ import java.io.File import org.apache.hadoop.util.VersionInfo -import org.apache.spark.{Logging, SparkFunSuite} +import org.apache.spark.{Logging, SparkConf, SparkFunSuite} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal, NamedExpression} import org.apache.spark.sql.catalyst.util.quietly @@ -39,6 +39,8 @@ import org.apache.spark.util.Utils @ExtendedHiveTest class VersionsSuite extends SparkFunSuite with Logging { + private val sparkConf = new SparkConf() + // In order to speed up test execution during development or in Jenkins, you can specify the path // of an existing Ivy cache: private val ivyPath: Option[String] = { @@ -59,6 +61,7 @@ class VersionsSuite extends SparkFunSuite with Logging { val badClient = IsolatedClientLoader.forVersion( hiveMetastoreVersion = HiveContext.hiveExecutionVersion, hadoopVersion = VersionInfo.getVersion, + sparkConf = sparkConf, config = buildConf(), ivyPath = ivyPath).createClient() val db = new CatalogDatabase("default", "desc", "loc", Map()) @@ -93,6 +96,7 @@ class VersionsSuite extends SparkFunSuite with Logging { IsolatedClientLoader.forVersion( hiveMetastoreVersion = "13", hadoopVersion = VersionInfo.getVersion, + sparkConf = sparkConf, config = buildConf(), ivyPath = ivyPath).createClient() } @@ -112,6 +116,7 @@ class VersionsSuite extends SparkFunSuite with Logging { IsolatedClientLoader.forVersion( hiveMetastoreVersion = version, hadoopVersion = VersionInfo.getVersion, + sparkConf = sparkConf, config = buildConf(), ivyPath = ivyPath).createClient() } |