Skip to content

Commit

Permalink
Using test UGI to mimic current user and groups
Browse files Browse the repository at this point in the history
Change-Id: Iafc17472e44a511402a3faa5e1889fa445b3c386
  • Loading branch information
jerryshao committed Apr 19, 2017
1 parent 4d3838a commit ab5117b
Showing 1 changed file with 53 additions and 31 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,10 @@

package org.apache.spark.deploy

import java.security.PrivilegedExceptionAction

import scala.util.Random

import org.apache.hadoop.fs.FileStatus
import org.apache.hadoop.fs.permission.{FsAction, FsPermission}
import org.apache.hadoop.security.UserGroupInformation
Expand All @@ -30,37 +34,55 @@ class SparkHadoopUtilSuite extends SparkFunSuite with Matchers {
val user = UserGroupInformation.getCurrentUser.getShortUserName
val groups = UserGroupInformation.getCurrentUser.getGroupNames
require(!groups.isEmpty)
val sparkHadoopUtil = new SparkHadoopUtil

// If file is owned by user and user has access permission
var status = fileStatus(user, groups.head, READ_WRITE, READ_WRITE, NONE)
sparkHadoopUtil.checkAccessPermission(status, READ) should be (true)
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be (true)

// If file is owned by user but user has no access permission
status = fileStatus(user, groups.head, NONE, READ_WRITE, NONE)
sparkHadoopUtil.checkAccessPermission(status, READ) should be (false)
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be (false)

// If file is owned by user's group and user's group has access permission
status = fileStatus("test", groups.head, NONE, READ_WRITE, NONE)
sparkHadoopUtil.checkAccessPermission(status, READ) should be (true)
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be (true)

// If file is owned by user's group but user's group has no access permission
status = fileStatus("test", groups.head, READ_WRITE, NONE, NONE)
sparkHadoopUtil.checkAccessPermission(status, READ) should be (false)
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be (false)

// If file is owned by other user and this user has access permission
status = fileStatus("test", "test", READ_WRITE, READ_WRITE, READ_WRITE)
sparkHadoopUtil.checkAccessPermission(status, READ) should be (true)
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be (true)

// If file is owned by other user but this user has no access permission
status = fileStatus("test", "test", READ_WRITE, READ_WRITE, NONE)
sparkHadoopUtil.checkAccessPermission(status, READ) should be (false)
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be (false)

val testUser = user + "-" + Random.nextInt(100)
val testGroups = groups.map { g => g + "-" + Random.nextInt(100) }
val testUgi = UserGroupInformation.createUserForTesting(testUser, testGroups)

testUgi.doAs(new PrivilegedExceptionAction[Void] {
override def run(): Void = {
val sparkHadoopUtil = new SparkHadoopUtil

// If file is owned by user and user has access permission
var status = fileStatus(testUser, testGroups.head, READ_WRITE, READ_WRITE, NONE)
sparkHadoopUtil.checkAccessPermission(status, READ) should be(true)
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(true)

// If file is owned by user but user has no access permission
status = fileStatus(testUser, testGroups.head, NONE, READ_WRITE, NONE)
sparkHadoopUtil.checkAccessPermission(status, READ) should be(false)
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(false)

var otherUser = "test"
var otherGroup = "test"
while (otherUser == testUser || testGroups.contains(otherGroup)) {
otherUser = s"test-${Random.nextInt(100)}"
otherGroup = s"test-${Random.nextInt(100)}"
}

// If file is owned by user's group and user's group has access permission
status = fileStatus(otherUser, testGroups.head, NONE, READ_WRITE, NONE)
sparkHadoopUtil.checkAccessPermission(status, READ) should be(true)
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(true)

// If file is owned by user's group but user's group has no access permission
status = fileStatus(otherUser, testGroups.head, READ_WRITE, NONE, NONE)
sparkHadoopUtil.checkAccessPermission(status, READ) should be(false)
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(false)

// If file is owned by other user and this user has access permission
status = fileStatus(otherUser, otherGroup, READ_WRITE, READ_WRITE, READ_WRITE)
sparkHadoopUtil.checkAccessPermission(status, READ) should be(true)
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(true)

// If file is owned by other user but this user has no access permission
status = fileStatus(otherUser, otherGroup, READ_WRITE, READ_WRITE, NONE)
sparkHadoopUtil.checkAccessPermission(status, READ) should be(false)
sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(false)

null
}
})
}

private def fileStatus(
Expand Down

0 comments on commit ab5117b

Please sign in to comment.