Skip to content

Commit 7e2f90d

Browse files
author
Michael Gummelt
committed
style
1 parent 4d57f7b commit 7e2f90d

File tree

8 files changed

+29
-33
lines changed

8 files changed

+29
-33
lines changed

core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -47,7 +47,7 @@ private[security] class HiveCredentialProvider extends HadoopDelegationTokenProv
4747
case NonFatal(e) =>
4848
logDebug("Fail to create Hive Configuration", e)
4949
hadoopConf
50-
case e: ClassNotFoundException =>
50+
case e: NoClassDefFoundError =>
5151
logError(classNotFoundErrorStr)
5252
throw e
5353
}
@@ -95,7 +95,7 @@ private[security] class HiveCredentialProvider extends HadoopDelegationTokenProv
9595

9696
None
9797
} catch {
98-
case e: ClassNotFoundException =>
98+
case e: NoClassDefFoundError =>
9999
logError(classNotFoundErrorStr)
100100
throw e
101101
}

core/src/test/scala/org/apache/spark/deploy/security/ConfigurableCredentialManagerSuite.scala

Lines changed: 3 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -81,9 +81,7 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers {
8181
val creds = new Credentials()
8282

8383
// Tokens cannot be obtained from HDFS, Hive, HBase in unit tests.
84-
credentialManager.obtainCredentials(
85-
hadoopConf,
86-
creds)
84+
credentialManager.obtainCredentials(hadoopConf, creds)
8785
val tokens = creds.getAllTokens
8886
tokens.size() should be (0)
8987
}
@@ -96,9 +94,7 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers {
9694

9795
val hiveCredentialProvider = new HiveCredentialProvider()
9896
val credentials = new Credentials()
99-
hiveCredentialProvider.obtainCredentials(
100-
hadoopConf,
101-
credentials)
97+
hiveCredentialProvider.obtainCredentials(hadoopConf, credentials)
10298

10399
credentials.getAllTokens.size() should be (0)
104100
}
@@ -109,9 +105,7 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers {
109105

110106
val hbaseTokenProvider = new HBaseCredentialProvider()
111107
val creds = new Credentials()
112-
hbaseTokenProvider.obtainCredentials(
113-
hadoopConf,
114-
creds)
108+
hbaseTokenProvider.obtainCredentials(hadoopConf, creds)
115109

116110
creds.getAllTokens.size should be (0)
117111
}

resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -250,7 +250,7 @@ private[spark] class ApplicationMaster(
250250
val credentialManager = new YARNConfigurableCredentialManager(
251251
sparkConf,
252252
yarnConf,
253-
YarnSparkHadoopUtil.get.yarnHadoopFSsToAccess(sparkConf, yarnConf))
253+
YarnSparkHadoopUtil.get.hadoopFSsToAccess(sparkConf, yarnConf))
254254

255255
val credentialRenewer = new AMCredentialRenewer(sparkConf, yarnConf, credentialManager)
256256
credentialRenewer.scheduleLoginFromKeytab()

resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -124,7 +124,7 @@ private[spark] class Client(
124124
private val credentialManager = new YARNConfigurableCredentialManager(
125125
sparkConf,
126126
hadoopConf,
127-
YarnSparkHadoopUtil.get.yarnHadoopFSsToAccess(sparkConf, hadoopConf))
127+
YarnSparkHadoopUtil.get.hadoopFSsToAccess(sparkConf, hadoopConf))
128128

129129
def reportLauncherState(state: SparkAppHandle.State): Unit = {
130130
launcherBackend.setState(state)
@@ -371,7 +371,7 @@ private[spark] class Client(
371371
val fs = destDir.getFileSystem(hadoopConf)
372372

373373
// Merge credentials obtained from registered providers
374-
val nearestTimeOfNextRenewal = credentialManager.obtainYARNCredentials(
374+
val nearestTimeOfNextRenewal = credentialManager.obtainCredentials(
375375
hadoopConf,
376376
credentials)
377377

resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -95,7 +95,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil {
9595
val credentialManager = new YARNConfigurableCredentialManager(
9696
sparkConf,
9797
hadoopConf,
98-
YarnSparkHadoopUtil.get.yarnHadoopFSsToAccess(sparkConf, hadoopConf))
98+
YarnSparkHadoopUtil.get.hadoopFSsToAccess(sparkConf, hadoopConf))
9999
credentialUpdater = new CredentialUpdater(sparkConf, hadoopConf, credentialManager)
100100
credentialUpdater.start()
101101
}
@@ -113,9 +113,9 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil {
113113
}
114114

115115
/** The filesystems for which YARN should fetch delegation tokens. */
116-
private[spark] def yarnHadoopFSsToAccess(
117-
sparkConf: SparkConf,
118-
hadoopConf: Configuration): Set[FileSystem] = {
116+
private[spark] def hadoopFSsToAccess(
117+
sparkConf: SparkConf,
118+
hadoopConf: Configuration): Set[FileSystem] = {
119119
val filesystemsToAccess = sparkConf.get(FILESYSTEMS_TO_ACCESS)
120120
.map(new Path(_).getFileSystem(hadoopConf))
121121
.toSet

resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -174,7 +174,7 @@ private[yarn] class AMCredentialRenewer(
174174
keytabLoggedInUGI.doAs(new PrivilegedExceptionAction[Void] {
175175
// Get a copy of the credentials
176176
override def run(): Void = {
177-
nearestNextRenewalTime = credentialManager.obtainYARNCredentials(
177+
nearestNextRenewalTime = credentialManager.obtainCredentials(
178178
freshHadoopConf,
179179
tempCreds)
180180
null

resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManager.scala

Lines changed: 5 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,6 @@ import org.apache.spark.deploy.security.ConfigurableCredentialManager
3030
import org.apache.spark.internal.Logging
3131
import org.apache.spark.util.Utils
3232

33-
3433
/**
3534
* This class loads credential providers registered under the YARN-specific
3635
* [[ServiceCredentialProvider]] interface, as well as the builtin credential providers defined
@@ -47,9 +46,9 @@ private[yarn] class YARNConfigurableCredentialManager(
4746
// public for testing
4847
val credentialProviders = getCredentialProviders
4948

50-
def obtainYARNCredentials(
51-
hadoopConf: Configuration,
52-
creds: Credentials): Long = {
49+
def obtainCredentials(
50+
hadoopConf: Configuration,
51+
creds: Credentials): Long = {
5352

5453
val superInterval = configurableCredentialManager.obtainCredentials(
5554
hadoopConf,
@@ -66,8 +65,7 @@ private[yarn] class YARNConfigurableCredentialManager(
6665
}.foldLeft(superInterval)(math.min)
6766
}
6867

69-
private def getCredentialProviders:
70-
Map[String, ServiceCredentialProvider] = {
68+
private def getCredentialProviders: Map[String, ServiceCredentialProvider] = {
7169
val providers = loadCredentialProviders
7270

7371
providers.
@@ -76,8 +74,7 @@ private[yarn] class YARNConfigurableCredentialManager(
7674
.toMap
7775
}
7876

79-
private def loadCredentialProviders:
80-
List[ServiceCredentialProvider] = {
77+
private def loadCredentialProviders: List[ServiceCredentialProvider] = {
8178
ServiceLoader.load(
8279
classOf[ServiceCredentialProvider],
8380
Utils.getContextOrSparkClassLoader)

resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala

Lines changed: 11 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -24,8 +24,7 @@ import org.scalatest.Matchers
2424
import org.apache.spark.{SparkConf, SparkFunSuite}
2525
import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil
2626

27-
class YARNConfigurableCredentialManagerSuite
28-
extends SparkFunSuite with Matchers {
27+
class YARNConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers {
2928
private var credentialManager: YARNConfigurableCredentialManager = null
3029
private var sparkConf: SparkConf = null
3130
private var hadoopConf: Configuration = null
@@ -39,11 +38,17 @@ class YARNConfigurableCredentialManagerSuite
3938
hadoopConf = new Configuration()
4039
}
4140

41+
override def afterAll(): Unit = {
42+
super.afterAll()
43+
44+
System.clearProperty("SPARK_YARN_MODE")
45+
}
46+
4247
test("Correctly loads credential providers") {
4348
credentialManager = new YARNConfigurableCredentialManager(
4449
sparkConf,
4550
hadoopConf,
46-
YarnSparkHadoopUtil.get.yarnHadoopFSsToAccess(sparkConf, hadoopConf))
51+
YarnSparkHadoopUtil.get.hadoopFSsToAccess(sparkConf, hadoopConf))
4752

4853
credentialManager.credentialProviders.get("yarn-test") should not be (None)
4954
}
@@ -55,7 +60,7 @@ class YARNTestCredentialProvider extends ServiceCredentialProvider {
5560
override def credentialsRequired(conf: Configuration): Boolean = true
5661

5762
override def obtainCredentials(
58-
hadoopConf: Configuration,
59-
sparkConf: SparkConf,
60-
creds: Credentials): Option[Long] = None
63+
hadoopConf: Configuration,
64+
sparkConf: SparkConf,
65+
creds: Credentials): Option[Long] = None
6166
}

0 commit comments

Comments
 (0)