Skip to content

Commit f4357e8

Browse files
committed
Address the comments
Change-Id: Id11665cfac20c5112b52868ebbc67d40f676f528
1 parent c0c1933 commit f4357e8

File tree

2 files changed

+100
-40
lines changed

2 files changed

+100
-40
lines changed

core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala

Lines changed: 10 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -97,6 +97,13 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
9797
.map { d => Utils.resolveURI(d).toString }
9898
.getOrElse(DEFAULT_LOG_DIR)
9999

100+
private val HISTORY_UI_ACLS_ENABLE = conf.getBoolean("spark.history.ui.acls.enable", false)
101+
private val HISTORY_UI_ADMIN_ACLS = conf.get("spark.history.ui.admin.acls", "")
102+
private val HISTORY_UI_ADMIN_ACLS_GROUPS = conf.get("spark.history.ui.admin.acls.groups", "")
103+
logInfo(s"History server ui acls " + (if (HISTORY_UI_ACLS_ENABLE) "enabled" else "disabled") +
104+
"; users with admin permissions: " + HISTORY_UI_ADMIN_ACLS.toString +
105+
"; groups with admin permissions" + HISTORY_UI_ADMIN_ACLS_GROUPS.toString)
106+
100107
private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf)
101108
private val fs = Utils.getHadoopFileSystem(logDir, hadoopConf)
102109

@@ -250,14 +257,12 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
250257
val appListener = replay(fileStatus, isApplicationCompleted(fileStatus), replayBus)
251258

252259
if (appListener.appId.isDefined) {
253-
val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false)
254-
ui.getSecurityManager.setAcls(uiAclsEnabled)
260+
ui.getSecurityManager.setAcls(HISTORY_UI_ACLS_ENABLE)
255261
// make sure to set admin acls before view acls so they are properly picked up
256-
val adminAcls = conf.get("spark.history.ui.admin.acls", "") + "," +
257-
appListener.adminAcls.getOrElse("")
262+
val adminAcls = HISTORY_UI_ADMIN_ACLS + "," + appListener.adminAcls.getOrElse("")
258263
ui.getSecurityManager.setAdminAcls(adminAcls)
259264
ui.getSecurityManager.setViewAcls(attempt.sparkUser, appListener.viewAcls.getOrElse(""))
260-
val adminAclsGroups = conf.get("spark.history.ui.admin.acls.groups", "") + "," +
265+
val adminAclsGroups = HISTORY_UI_ADMIN_ACLS_GROUPS + "," +
261266
appListener.adminAclsGroups.getOrElse("")
262267
ui.getSecurityManager.setAdminAclsGroups(adminAclsGroups)
263268
ui.getSecurityManager.setViewAclsGroups(appListener.viewAclsGroups.getOrElse(""))

core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala

Lines changed: 90 additions & 35 deletions
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ import org.scalatest.BeforeAndAfter
3535
import org.scalatest.Matchers
3636
import org.scalatest.concurrent.Eventually._
3737

38-
import org.apache.spark.{SparkConf, SparkFunSuite}
38+
import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite}
3939
import org.apache.spark.internal.Logging
4040
import org.apache.spark.io._
4141
import org.apache.spark.scheduler._
@@ -475,47 +475,101 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
475475
}
476476
}
477477

478-
479478
test("support history server ui admin acls") {
480-
val conf = createTestConf()
479+
def createAndCheck(conf: SparkConf, properties: (String, String)*)
480+
(checkFn: SecurityManager => Unit): Unit = {
481+
// Empty the testDir for each test.
482+
if (testDir.exists() && testDir.isDirectory) {
483+
testDir.listFiles().foreach { f => if (f.isFile) f.delete() }
484+
}
485+
486+
var provider: FsHistoryProvider = null
487+
try {
488+
provider = new FsHistoryProvider(conf)
489+
val log = newLogFile("app1", Some("attempt1"), inProgress = false)
490+
writeFile(log, true, None,
491+
SparkListenerApplicationStart("app1", Some("app1"), System.currentTimeMillis(),
492+
"test", Some("attempt1")),
493+
SparkListenerEnvironmentUpdate(Map(
494+
"Spark Properties" -> properties.toSeq,
495+
"JVM Information" -> Seq.empty,
496+
"System Properties" -> Seq.empty,
497+
"Classpath Entries" -> Seq.empty
498+
)),
499+
SparkListenerApplicationEnd(System.currentTimeMillis()))
500+
501+
provider.checkForLogs()
502+
val appUi = provider.getAppUI("app1", Some("attempt1"))
503+
504+
assert(appUi.nonEmpty)
505+
val securityManager = appUi.get.ui.securityManager
506+
checkFn(securityManager)
507+
} finally {
508+
if (provider != null) {
509+
provider.stop()
510+
}
511+
}
512+
}
513+
514+
// Test both history ui admin acls and application acls are configured.
515+
val conf1 = createTestConf()
481516
.set("spark.history.ui.acls.enable", "true")
482517
.set("spark.history.ui.admin.acls", "user1,user2")
483518
.set("spark.history.ui.admin.acls.groups", "group1")
484519
.set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName)
485520

486-
val provider = new FsHistoryProvider(conf)
487-
488-
val log = newLogFile("app1", Some("attempt1"), inProgress = false)
489-
writeFile(log, true, None,
490-
SparkListenerApplicationStart("app1", Some("app1"), System.currentTimeMillis(),
491-
"test", Some("attempt1")),
492-
SparkListenerEnvironmentUpdate(Map(
493-
"Spark Properties" -> Seq(
494-
("spark.admin.acls", "user"),
495-
("spark.admin.acls.groups", "group")),
496-
"JVM Information" -> Seq.empty,
497-
"System Properties" -> Seq.empty,
498-
"Classpath Entries" -> Seq.empty
499-
)),
500-
SparkListenerApplicationEnd(System.currentTimeMillis()))
501-
502-
provider.checkForLogs()
503-
val appUi = provider.getAppUI("app1", Some("attempt1"))
504-
505-
assert (appUi.nonEmpty)
506-
val securityManager = appUi.get.ui.securityManager
521+
createAndCheck(conf1, ("spark.admin.acls", "user"), ("spark.admin.acls.groups", "group")) {
522+
securityManager =>
523+
// Test whether user has permission to access UI.
524+
securityManager.checkUIViewPermissions("user1") should be (true)
525+
securityManager.checkUIViewPermissions("user2") should be (true)
526+
securityManager.checkUIViewPermissions("user") should be (true)
527+
securityManager.checkUIViewPermissions("abc") should be (false)
528+
529+
// Test whether user with admin group has permission to access UI.
530+
securityManager.checkUIViewPermissions("user3") should be (true)
531+
securityManager.checkUIViewPermissions("user4") should be (true)
532+
securityManager.checkUIViewPermissions("user5") should be (true)
533+
securityManager.checkUIViewPermissions("user6") should be (false)
534+
}
507535

508-
// Test whether user has permission to access UI.
509-
securityManager.checkUIViewPermissions("user1") should be (true)
510-
securityManager.checkUIViewPermissions("user2") should be (true)
511-
securityManager.checkUIViewPermissions("user") should be (true)
512-
securityManager.checkUIViewPermissions("abc") should be (false)
536+
// Test only history ui admin acls are configured.
537+
val conf2 = createTestConf()
538+
.set("spark.history.ui.acls.enable", "true")
539+
.set("spark.history.ui.admin.acls", "user1,user2")
540+
.set("spark.history.ui.admin.acls.groups", "group1")
541+
.set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName)
542+
createAndCheck(conf2) { securityManager =>
543+
// Test whether user has permission to access UI.
544+
securityManager.checkUIViewPermissions("user1") should be (true)
545+
securityManager.checkUIViewPermissions("user2") should be (true)
546+
// Check the unknown "user" should return false
547+
securityManager.checkUIViewPermissions("user") should be (false)
548+
549+
// Test whether user with admin group has permission to access UI.
550+
securityManager.checkUIViewPermissions("user3") should be (true)
551+
securityManager.checkUIViewPermissions("user4") should be (true)
552+
// Check the "user5" without mapping relation should return false
553+
securityManager.checkUIViewPermissions("user5") should be (false)
554+
}
513555

514-
// Test whether user with admin group has permission to access UI.
515-
securityManager.checkUIViewPermissions("user3") should be (true)
516-
securityManager.checkUIViewPermissions("user4") should be (true)
517-
securityManager.checkUIViewPermissions("user5") should be (false)
518-
}
556+
// Test neither history ui admin acls nor application acls are configured.
557+
val conf3 = createTestConf()
558+
.set("spark.history.ui.acls.enable", "true")
559+
.set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName)
560+
createAndCheck(conf3) { securityManager =>
561+
// Test whether user has permission to access UI.
562+
securityManager.checkUIViewPermissions("user1") should be (false)
563+
securityManager.checkUIViewPermissions("user2") should be (false)
564+
securityManager.checkUIViewPermissions("user") should be (false)
565+
566+
// Test whether user with admin group has permission to access UI.
567+
// Check should be failed since we don't have acl group settings.
568+
securityManager.checkUIViewPermissions("user3") should be (false)
569+
securityManager.checkUIViewPermissions("user4") should be (false)
570+
securityManager.checkUIViewPermissions("user5") should be (false)
571+
}
572+
}
519573

520574
/**
521575
* Asks the provider to check for logs and calls a function to perform checks on the updated
@@ -579,7 +633,8 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
579633
class TestGroupsMappingProvider extends GroupMappingServiceProvider {
580634
private val mappings = Map(
581635
"user3" -> "group1",
582-
"user4" -> "group1")
636+
"user4" -> "group1",
637+
"user5" -> "group")
583638

584639
override def getGroups(username: String): Set[String] = {
585640
mappings.get(username).map(Set(_)).getOrElse(Set.empty)

0 commit comments

Comments
 (0)