aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.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/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.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/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala')
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala28
1 files changed, 11 insertions, 17 deletions
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
index 9012736bc2..f4235df245 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
@@ -27,7 +27,8 @@ import scala.xml.Node
import com.google.common.io.ByteStreams
import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder}
-import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
+import org.apache.hadoop.fs.{FileStatus, Path}
+import org.apache.hadoop.fs.permission.FsAction
import org.apache.hadoop.hdfs.DistributedFileSystem
import org.apache.hadoop.hdfs.protocol.HdfsConstants
import org.apache.hadoop.security.AccessControlException
@@ -318,21 +319,14 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
// scan for modified applications, replay and merge them
val logInfos: Seq[FileStatus] = statusList
.filter { entry =>
- try {
- val prevFileSize = fileToAppInfo.get(entry.getPath()).map{_.fileSize}.getOrElse(0L)
- !entry.isDirectory() &&
- // FsHistoryProvider generates a hidden file which can't be read. Accidentally
- // reading a garbage file is safe, but we would log an error which can be scary to
- // the end-user.
- !entry.getPath().getName().startsWith(".") &&
- prevFileSize < entry.getLen()
- } catch {
- case e: AccessControlException =>
- // Do not use "logInfo" since these messages can get pretty noisy if printed on
- // every poll.
- logDebug(s"No permission to read $entry, ignoring.")
- false
- }
+ val prevFileSize = fileToAppInfo.get(entry.getPath()).map{_.fileSize}.getOrElse(0L)
+ !entry.isDirectory() &&
+ // FsHistoryProvider generates a hidden file which can't be read. Accidentally
+ // reading a garbage file is safe, but we would log an error which can be scary to
+ // the end-user.
+ !entry.getPath().getName().startsWith(".") &&
+ prevFileSize < entry.getLen() &&
+ SparkHadoopUtil.get.checkAccessPermission(entry, FsAction.READ)
}
.flatMap { entry => Some(entry) }
.sortWith { case (entry1, entry2) =>
@@ -445,7 +439,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
/**
* Replay the log files in the list and merge the list of old applications with new ones
*/
- private def mergeApplicationListing(fileStatus: FileStatus): Unit = {
+ protected def mergeApplicationListing(fileStatus: FileStatus): Unit = {
val newAttempts = try {
val eventsFilter: ReplayEventsFilter = { eventString =>
eventString.startsWith(APPL_START_EVENT_PREFIX) ||