Skip to content

Commit 4d3838a

Browse files
committed
Address the comments to change the UT
Change-Id: Ib9ac4be0a896531a529c260197431df1c3adf77a
1 parent 9447011 commit 4d3838a

File tree

2 files changed

+84
-16
lines changed

2 files changed

+84
-16
lines changed
Lines changed: 83 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,83 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.deploy
19+
20+
import org.apache.hadoop.fs.FileStatus
21+
import org.apache.hadoop.fs.permission.{FsAction, FsPermission}
22+
import org.apache.hadoop.security.UserGroupInformation
23+
import org.scalatest.Matchers
24+
25+
import org.apache.spark.SparkFunSuite
26+
27+
class SparkHadoopUtilSuite extends SparkFunSuite with Matchers {
28+
test("check file permission") {
29+
import FsAction._
30+
val user = UserGroupInformation.getCurrentUser.getShortUserName
31+
val groups = UserGroupInformation.getCurrentUser.getGroupNames
32+
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)
64+
}
65+
66+
private def fileStatus(
67+
owner: String,
68+
group: String,
69+
userAction: FsAction,
70+
groupAction: FsAction,
71+
otherAction: FsAction): FileStatus = {
72+
new FileStatus(0L,
73+
false,
74+
0,
75+
0L,
76+
0L,
77+
0L,
78+
new FsPermission(userAction, groupAction, otherAction),
79+
owner,
80+
group,
81+
null)
82+
}
83+
}

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

Lines changed: 1 addition & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -27,8 +27,7 @@ import scala.concurrent.duration._
2727
import scala.language.postfixOps
2828

2929
import com.google.common.io.{ByteStreams, Files}
30-
import org.apache.hadoop.fs.{FileStatus, Path}
31-
import org.apache.hadoop.fs.permission.FsAction
30+
import org.apache.hadoop.fs.FileStatus
3231
import org.apache.hadoop.hdfs.DistributedFileSystem
3332
import org.json4s.jackson.JsonMethods._
3433
import org.mockito.Matchers.any
@@ -38,7 +37,6 @@ import org.scalatest.Matchers
3837
import org.scalatest.concurrent.Eventually._
3938

4039
import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite}
41-
import org.apache.spark.deploy.SparkHadoopUtil
4240
import org.apache.spark.internal.Logging
4341
import org.apache.spark.io._
4442
import org.apache.spark.scheduler._
@@ -156,20 +154,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
156154
SparkListenerApplicationStart("app1-2", Some("app1-2"), 1L, "test", None),
157155
SparkListenerApplicationEnd(2L)
158156
)
159-
160-
val path = new Path(logFile2.toURI)
161-
val fs = path.getFileSystem(SparkHadoopUtil.get.conf)
162-
val status = fs.getFileStatus(path)
163-
SparkHadoopUtil.get.checkAccessPermission(status, FsAction.READ) should be (true)
164-
165157
logFile2.setReadable(false, false)
166-
val status1 = fs.getFileStatus(path)
167-
SparkHadoopUtil.get.checkAccessPermission(status1, FsAction.READ) should be (false)
168-
169-
logFile2.setReadable(false, true)
170-
val status2 = fs.getFileStatus(path)
171-
SparkHadoopUtil.get.checkAccessPermission(status2, FsAction.READ) should be (false)
172-
173158

174159
updateAndCheck(provider) { list =>
175160
list.size should be (1)

0 commit comments

Comments
 (0)