Skip to content

Commit ad4e33b

Browse files
author
Michael Gummelt
committed
add tests
1 parent 810c6b2 commit ad4e33b

File tree

8 files changed

+62
-55
lines changed

8 files changed

+62
-55
lines changed
Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,3 @@
1+
org.apache.spark.deploy.security.HadoopFSCredentialProvider
2+
org.apache.spark.deploy.security.HBaseCredentialProvider
3+
org.apache.spark.deploy.security.HiveCredentialProvider

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

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,8 +20,9 @@ package org.apache.spark.deploy.security
2020
import org.apache.hadoop.conf.Configuration
2121
import org.apache.hadoop.fs.Path
2222

23-
/** [[HadoopAccessManager]] returns information related to how Hadoop delegation tokens should be
24-
* fetched.
23+
/**
24+
* Methods in [[HadoopAccessManager]] return scheduler-specific information related to how Hadoop
25+
* delegation tokens should be fetched.
2526
*/
2627
private[spark] trait HadoopAccessManager {
2728

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

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -17,11 +17,13 @@
1717

1818
package org.apache.spark.deploy.security
1919

20+
import org.scalatest.{BeforeAndAfter, Matchers}
21+
2022
import org.apache.hadoop.conf.Configuration
2123
import org.apache.hadoop.io.Text
2224
import org.apache.hadoop.security.{Credentials, UserGroupInformation}
2325
import org.apache.hadoop.security.token.Token
24-
import org.scalatest.{BeforeAndAfter, Matchers}
26+
2527
import org.apache.spark.{SparkConf, SparkFunSuite}
2628

2729
class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfter {
@@ -136,9 +138,9 @@ class TestCredentialProvider extends ServiceCredentialProvider {
136138
override def credentialsRequired(conf: Configuration): Boolean = true
137139

138140
override def obtainCredentials(
139-
hadoopConf: Configuration,
140-
hadoopAccessManager: HadoopAccessManager,
141-
creds: Credentials): Option[Long] = {
141+
hadoopConf: Configuration,
142+
hadoopAccessManager: HadoopAccessManager,
143+
creds: Credentials): Option[Long] = {
142144
if (creds == null) {
143145
// Guard out other unit test failures.
144146
return None

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

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -53,9 +53,9 @@ trait ServiceCredentialProvider {
5353
* renewal, otherwise None should be returned.
5454
*/
5555
def obtainCredentials(
56-
hadoopConf: Configuration,
57-
sparkConf: SparkConf,
58-
creds: Credentials): Option[Long]
56+
hadoopConf: Configuration,
57+
sparkConf: SparkConf,
58+
creds: Credentials): Option[Long]
5959
}
6060

6161

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,7 @@ private[yarn] class YARNConfigurableCredentialManager(
4040
hadoopConf,
4141
new YARNHadoopAccessManager(hadoopConf, sparkConf)) {
4242

43-
private val deprecatedCredentialProviders = getDeprecatedCredentialProviders
43+
val deprecatedCredentialProviders = getDeprecatedCredentialProviders
4444

4545
def getDeprecatedCredentialProviders:
4646
Map[String, org.apache.spark.deploy.yarn.security.ServiceCredentialProvider] = {
Original file line numberDiff line numberDiff line change
@@ -0,0 +1 @@
1+
org.apache.spark.deploy.yarn.security.YARNTestCredentialProvider

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

Lines changed: 33 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -18,12 +18,15 @@
1818
package org.apache.spark.deploy.yarn.security
1919

2020
import org.apache.hadoop.conf.Configuration
21+
import org.apache.hadoop.io.Text
22+
import org.apache.hadoop.security.Credentials
23+
import org.apache.hadoop.security.token.Token
2124
import org.scalatest.{BeforeAndAfter, Matchers}
2225

2326
import org.apache.spark.{SparkConf, SparkFunSuite}
24-
import org.apache.spark.deploy.security.ConfigurableCredentialManager
2527

26-
class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfter {
28+
class YARNConfigurableCredentialManagerSuite
29+
extends SparkFunSuite with Matchers with BeforeAndAfter {
2730
private var credentialManager: YARNConfigurableCredentialManager = null
2831
private var sparkConf: SparkConf = null
2932
private var hadoopConf: Configuration = null
@@ -33,21 +36,39 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit
3336

3437
sparkConf = new SparkConf()
3538
hadoopConf = new Configuration()
36-
System.setProperty("SPARK_YARN_MODE", "true")
3739
}
3840

39-
override def afterAll(): Unit = {
40-
System.clearProperty("SPARK_YARN_MODE")
41+
test("Correctly loads deprecated credential providers") {
42+
credentialManager = new YARNConfigurableCredentialManager(sparkConf, hadoopConf)
4143

42-
super.afterAll()
44+
credentialManager.deprecatedCredentialProviders.get("yarn-test") should not be (None)
4345
}
46+
}
4447

45-
test("Correctly load ") {
46-
credentialManager = new YARNConfigurableCredentialManager(sparkConf, hadoopConf)
48+
class YARNTestCredentialProvider extends ServiceCredentialProvider {
49+
val tokenRenewalInterval = 86400 * 1000L
50+
var timeOfNextTokenRenewal = 0L
51+
52+
override def serviceName: String = "yarn-test"
53+
54+
override def credentialsRequired(conf: Configuration): Boolean = true
55+
56+
override def obtainCredentials(
57+
hadoopConf: Configuration,
58+
sparkConf: SparkConf,
59+
creds: Credentials): Option[Long] = {
60+
if (creds == null) {
61+
// Guard out other unit test failures.
62+
return None
63+
}
64+
65+
val emptyToken = new Token()
66+
emptyToken.setService(new Text(serviceName))
67+
creds.addToken(emptyToken.getService, emptyToken)
68+
69+
val currTime = System.currentTimeMillis()
70+
timeOfNextTokenRenewal = (currTime - currTime % tokenRenewalInterval) + tokenRenewalInterval
4771

48-
assert(credentialManager
49-
.getServiceCredentialProvider("hadoopfs")
50-
.get
51-
.isInstanceOf[YARNHadoopAccessManager])
72+
Some(timeOfNextTokenRenewal)
5273
}
5374
}

resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProviderSuite.scala renamed to resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManagerSuite.scala

Lines changed: 12 additions & 33 deletions
Original file line numberDiff line numberDiff line change
@@ -18,53 +18,32 @@
1818
package org.apache.spark.deploy.yarn.security
1919

2020
import org.apache.hadoop.conf.Configuration
21-
import org.scalatest.{Matchers, PrivateMethodTester}
21+
import org.scalatest.Matchers
2222

23-
import org.apache.spark.{SparkException, SparkFunSuite}
24-
import org.apache.spark.deploy.yarn.security.YARNHadoopAccessManager
23+
import org.apache.spark.{SparkConf, SparkException, SparkFunSuite}
2524

26-
class YARNHadoopFSCredentialProviderSuite
27-
extends SparkFunSuite
28-
with PrivateMethodTester
29-
with Matchers {
30-
private val _getTokenRenewer = PrivateMethod[String]('getTokenRenewer)
31-
32-
private def getTokenRenewer(
33-
fsCredentialProvider: YARNHadoopAccessManager, conf: Configuration): String = {
34-
fsCredentialProvider invokePrivate _getTokenRenewer(conf)
35-
}
36-
37-
private var hadoopFsCredentialProvider: YARNHadoopAccessManager = null
38-
39-
override def beforeAll() {
40-
super.beforeAll()
41-
42-
if (hadoopFsCredentialProvider == null) {
43-
hadoopFsCredentialProvider = new YARNHadoopAccessManager()
44-
}
45-
}
46-
47-
override def afterAll() {
48-
if (hadoopFsCredentialProvider != null) {
49-
hadoopFsCredentialProvider = null
50-
}
51-
52-
super.afterAll()
53-
}
25+
class YARNHadoopAccessManagerSuite extends SparkFunSuite with Matchers {
5426

5527
test("check token renewer") {
5628
val hadoopConf = new Configuration()
5729
hadoopConf.set("yarn.resourcemanager.address", "myrm:8033")
5830
hadoopConf.set("yarn.resourcemanager.principal", "yarn/myrm:[email protected]")
59-
val renewer = getTokenRenewer(hadoopFsCredentialProvider, hadoopConf)
31+
32+
val sparkConf = new SparkConf()
33+
val yarnHadoopAccessManager = new YARNHadoopAccessManager(hadoopConf, sparkConf)
34+
35+
val renewer = yarnHadoopAccessManager.getTokenRenewer
6036
renewer should be ("yarn/myrm:[email protected]")
6137
}
6238

6339
test("check token renewer default") {
6440
val hadoopConf = new Configuration()
41+
val sparkConf = new SparkConf()
42+
val yarnHadoopAccessManager = new YARNHadoopAccessManager(hadoopConf, sparkConf)
43+
6544
val caught =
6645
intercept[SparkException] {
67-
getTokenRenewer(hadoopFsCredentialProvider, hadoopConf)
46+
yarnHadoopAccessManager.getTokenRenewer
6847
}
6948
assert(caught.getMessage === "Can't get Master Kerberos principal for use as renewer")
7049
}

0 commit comments

Comments
 (0)