Skip to content

Commit a175cc1

Browse files
committed
Review fix.
1 parent da8ed8d commit a175cc1

File tree

4 files changed

+24
-1
lines changed

4 files changed

+24
-1
lines changed

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -249,7 +249,7 @@ private[spark] class HadoopDelegationTokenManager(
249249
private def loadProviders(): Map[String, HadoopDelegationTokenProvider] = {
250250
val loader = ServiceLoader.load(classOf[HadoopDelegationTokenProvider],
251251
Utils.getContextOrSparkClassLoader)
252-
var providers = mutable.ArrayBuffer[HadoopDelegationTokenProvider]()
252+
val providers = mutable.ArrayBuffer[HadoopDelegationTokenProvider]()
253253

254254
val iterator = loader.iterator
255255
while (iterator.hasNext) {
Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1 @@
1+
org.apache.spark.deploy.security.ExceptionThrowingDelegationTokenProvider

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

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,16 +18,36 @@
1818
package org.apache.spark.deploy.security
1919

2020
import org.apache.hadoop.conf.Configuration
21+
import org.apache.hadoop.fs.FileSystem
22+
import org.apache.hadoop.security.Credentials
2123

2224
import org.apache.spark.{SparkConf, SparkFunSuite}
2325

26+
private class ExceptionThrowingDelegationTokenProvider extends HadoopDelegationTokenProvider {
27+
throw new IllegalArgumentException
28+
29+
override def serviceName: String = "throw"
30+
31+
override def delegationTokensRequired(
32+
sparkConf: SparkConf,
33+
hadoopConf: Configuration): Boolean = throw new IllegalArgumentException
34+
35+
override def obtainDelegationTokens(
36+
hadoopConf: Configuration,
37+
sparkConf: SparkConf,
38+
fileSystems: Set[FileSystem],
39+
creds: Credentials): Option[Long] = throw new IllegalArgumentException
40+
}
41+
2442
class HadoopDelegationTokenManagerSuite extends SparkFunSuite {
2543
private val hadoopConf = new Configuration()
2644

2745
test("default configuration") {
2846
val manager = new HadoopDelegationTokenManager(new SparkConf(false), hadoopConf, null)
2947
assert(manager.isProviderLoaded("hadoopfs"))
3048
assert(manager.isProviderLoaded("hbase"))
49+
// This checks that providers are loaded independently and they have no effect on each other
50+
assert(!manager.isProviderLoaded("throw"))
3151
}
3252

3353
test("using deprecated configurations") {

sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -91,6 +91,8 @@ private object NoHiveTest {
9191
def runTest(): Unit = {
9292
try {
9393
val manager = new HadoopDelegationTokenManager(new SparkConf(), new Configuration(), null)
94+
assert(manager.isProviderLoaded("hadoopfs"))
95+
assert(manager.isProviderLoaded("hbase"))
9496
require(!manager.isProviderLoaded("hive"))
9597
} catch {
9698
case e: Throwable =>

0 commit comments

Comments
 (0)