aboutsummaryrefslogtreecommitdiff
path: root/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
diff options
context:
space:
mode:
authorjerryshao <sshao@hortonworks.com>2017-04-20 16:02:09 -0700
committerMarcelo Vanzin <vanzin@cloudera.com>2017-04-20 16:02:09 -0700
commit592f5c89349f3c5b6ec0531c6514b8f7d95ad8da (patch)
tree11a60e8bf35212ff0980dbe288fb687b71c6bbfd /core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
parent033206355339677812a250b2b64818a261871fd2 (diff)
downloadspark-592f5c89349f3c5b6ec0531c6514b8f7d95ad8da.tar.gz
spark-592f5c89349f3c5b6ec0531c6514b8f7d95ad8da.tar.bz2
spark-592f5c89349f3c5b6ec0531c6514b8f7d95ad8da.zip
[SPARK-20172][CORE] Add file permission check when listing files in FsHistoryProvider
## What changes were proposed in this pull request? In the current Spark's HistoryServer we expected to get `AccessControlException` during listing all the files, but unfortunately it was not worked because we actually doesn't check the access permission and no other calls will throw such exception. What was worse is that this check will be deferred until reading files, which is not necessary and quite verbose, since it will be printed out the exception in every 10 seconds when checking the files. So here with this fix, we actually check the read permission during listing the files, which could avoid unnecessary file read later on and suppress the verbose log. ## How was this patch tested? Add unit test to verify. Author: jerryshao <sshao@hortonworks.com> Closes #17495 from jerryshao/SPARK-20172.
Diffstat (limited to 'core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala')
-rw-r--r--core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala16
1 files changed, 14 insertions, 2 deletions
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
index ec580a44b8..456158d41b 100644
--- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
@@ -27,6 +27,7 @@ import scala.concurrent.duration._
import scala.language.postfixOps
import com.google.common.io.{ByteStreams, Files}
+import org.apache.hadoop.fs.FileStatus
import org.apache.hadoop.hdfs.DistributedFileSystem
import org.json4s.jackson.JsonMethods._
import org.mockito.Matchers.any
@@ -130,9 +131,19 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
}
}
- test("SPARK-3697: ignore directories that cannot be read.") {
+ test("SPARK-3697: ignore files that cannot be read.") {
// setReadable(...) does not work on Windows. Please refer JDK-6728842.
assume(!Utils.isWindows)
+
+ class TestFsHistoryProvider extends FsHistoryProvider(createTestConf()) {
+ var mergeApplicationListingCall = 0
+ override protected def mergeApplicationListing(fileStatus: FileStatus): Unit = {
+ super.mergeApplicationListing(fileStatus)
+ mergeApplicationListingCall += 1
+ }
+ }
+ val provider = new TestFsHistoryProvider
+
val logFile1 = newLogFile("new1", None, inProgress = false)
writeFile(logFile1, true, None,
SparkListenerApplicationStart("app1-1", Some("app1-1"), 1L, "test", None),
@@ -145,10 +156,11 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
)
logFile2.setReadable(false, false)
- val provider = new FsHistoryProvider(createTestConf())
updateAndCheck(provider) { list =>
list.size should be (1)
}
+
+ provider.mergeApplicationListingCall should be (1)
}
test("history file is renamed from inprogress to completed") {