Skip to content

Commit ab5117b

Browse files
committed
Using test UGI to mimic current user and groups
Change-Id: Iafc17472e44a511402a3faa5e1889fa445b3c386
1 parent 4d3838a commit ab5117b

File tree

1 file changed

+53
-31
lines changed

1 file changed

+53
-31
lines changed

core/src/test/scala/org/apache/spark/deploy/SparkHadoopUtilSuite.scala

Lines changed: 53 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,10 @@
1717

1818
package org.apache.spark.deploy
1919

20+
import java.security.PrivilegedExceptionAction
21+
22+
import scala.util.Random
23+
2024
import org.apache.hadoop.fs.FileStatus
2125
import org.apache.hadoop.fs.permission.{FsAction, FsPermission}
2226
import org.apache.hadoop.security.UserGroupInformation
@@ -30,37 +34,55 @@ class SparkHadoopUtilSuite extends SparkFunSuite with Matchers {
3034
val user = UserGroupInformation.getCurrentUser.getShortUserName
3135
val groups = UserGroupInformation.getCurrentUser.getGroupNames
3236
require(!groups.isEmpty)
33-
val sparkHadoopUtil = new SparkHadoopUtil
34-
35-
// If file is owned by user and user has access permission
36-
var status = fileStatus(user, groups.head, READ_WRITE, READ_WRITE, NONE)
37-
sparkHadoopUtil.checkAccessPermission(status, READ) should be (true)
38-
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be (true)
39-
40-
// If file is owned by user but user has no access permission
41-
status = fileStatus(user, groups.head, NONE, READ_WRITE, NONE)
42-
sparkHadoopUtil.checkAccessPermission(status, READ) should be (false)
43-
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be (false)
44-
45-
// If file is owned by user's group and user's group has access permission
46-
status = fileStatus("test", groups.head, NONE, READ_WRITE, NONE)
47-
sparkHadoopUtil.checkAccessPermission(status, READ) should be (true)
48-
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be (true)
49-
50-
// If file is owned by user's group but user's group has no access permission
51-
status = fileStatus("test", groups.head, READ_WRITE, NONE, NONE)
52-
sparkHadoopUtil.checkAccessPermission(status, READ) should be (false)
53-
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be (false)
54-
55-
// If file is owned by other user and this user has access permission
56-
status = fileStatus("test", "test", READ_WRITE, READ_WRITE, READ_WRITE)
57-
sparkHadoopUtil.checkAccessPermission(status, READ) should be (true)
58-
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be (true)
59-
60-
// If file is owned by other user but this user has no access permission
61-
status = fileStatus("test", "test", READ_WRITE, READ_WRITE, NONE)
62-
sparkHadoopUtil.checkAccessPermission(status, READ) should be (false)
63-
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be (false)
37+
38+
val testUser = user + "-" + Random.nextInt(100)
39+
val testGroups = groups.map { g => g + "-" + Random.nextInt(100) }
40+
val testUgi = UserGroupInformation.createUserForTesting(testUser, testGroups)
41+
42+
testUgi.doAs(new PrivilegedExceptionAction[Void] {
43+
override def run(): Void = {
44+
val sparkHadoopUtil = new SparkHadoopUtil
45+
46+
// If file is owned by user and user has access permission
47+
var status = fileStatus(testUser, testGroups.head, READ_WRITE, READ_WRITE, NONE)
48+
sparkHadoopUtil.checkAccessPermission(status, READ) should be(true)
49+
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(true)
50+
51+
// If file is owned by user but user has no access permission
52+
status = fileStatus(testUser, testGroups.head, NONE, READ_WRITE, NONE)
53+
sparkHadoopUtil.checkAccessPermission(status, READ) should be(false)
54+
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(false)
55+
56+
var otherUser = "test"
57+
var otherGroup = "test"
58+
while (otherUser == testUser || testGroups.contains(otherGroup)) {
59+
otherUser = s"test-${Random.nextInt(100)}"
60+
otherGroup = s"test-${Random.nextInt(100)}"
61+
}
62+
63+
// If file is owned by user's group and user's group has access permission
64+
status = fileStatus(otherUser, testGroups.head, NONE, READ_WRITE, NONE)
65+
sparkHadoopUtil.checkAccessPermission(status, READ) should be(true)
66+
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(true)
67+
68+
// If file is owned by user's group but user's group has no access permission
69+
status = fileStatus(otherUser, testGroups.head, READ_WRITE, NONE, NONE)
70+
sparkHadoopUtil.checkAccessPermission(status, READ) should be(false)
71+
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(false)
72+
73+
// If file is owned by other user and this user has access permission
74+
status = fileStatus(otherUser, otherGroup, READ_WRITE, READ_WRITE, READ_WRITE)
75+
sparkHadoopUtil.checkAccessPermission(status, READ) should be(true)
76+
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(true)
77+
78+
// If file is owned by other user but this user has no access permission
79+
status = fileStatus(otherUser, otherGroup, READ_WRITE, READ_WRITE, NONE)
80+
sparkHadoopUtil.checkAccessPermission(status, READ) should be(false)
81+
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(false)
82+
83+
null
84+
}
85+
})
6486
}
6587

6688
private def fileStatus(

0 commit comments

Comments
 (0)