Skip to content

Commit fb68910

Browse files
committed
[SPARK-24948][SHS] Delegate check access permissions to the file system
In `SparkHadoopUtil. checkAccessPermission`, we consider only basic permissions in order to check wether a user can access a file or not. This is not a complete check, as it ignores ACLs and other policies a file system may apply in its internal. So this can result in returning wrongly that a user cannot access a file (despite he actually can). The PR proposes to delegate to the filesystem the check whether a file is accessible or not, in order to return the right result. A caching layer is added for performance reasons. modified UTs Author: Marco Gaido <marcogaido91@gmail.com> Closes #21895 from mgaido91/SPARK-24948.
1 parent 136588e commit fb68910

4 files changed

Lines changed: 89 additions & 140 deletions

File tree

core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala

Lines changed: 0 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,6 @@ import scala.util.control.NonFatal
3131
import com.google.common.primitives.Longs
3232
import org.apache.hadoop.conf.Configuration
3333
import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter}
34-
import org.apache.hadoop.fs.permission.FsAction
3534
import org.apache.hadoop.mapred.JobConf
3635
import org.apache.hadoop.security.{Credentials, UserGroupInformation}
3736
import org.apache.hadoop.security.token.{Token, TokenIdentifier}
@@ -378,28 +377,6 @@ class SparkHadoopUtil extends Logging {
378377
buffer.toString
379378
}
380379

381-
private[spark] def checkAccessPermission(status: FileStatus, mode: FsAction): Boolean = {
382-
val perm = status.getPermission
383-
val ugi = UserGroupInformation.getCurrentUser
384-
385-
if (ugi.getShortUserName == status.getOwner) {
386-
if (perm.getUserAction.implies(mode)) {
387-
return true
388-
}
389-
} else if (ugi.getGroupNames.contains(status.getGroup)) {
390-
if (perm.getGroupAction.implies(mode)) {
391-
return true
392-
}
393-
} else if (perm.getOtherAction.implies(mode)) {
394-
return true
395-
}
396-
397-
logDebug(s"Permission denied: user=${ugi.getShortUserName}, " +
398-
s"path=${status.getPath}:${status.getOwner}:${status.getGroup}" +
399-
s"${if (status.isDirectory) "d" else "-"}$perm")
400-
false
401-
}
402-
403380
def serialize(creds: Credentials): Array[Byte] = {
404381
val byteStream = new ByteArrayOutputStream
405382
val dataStream = new DataOutputStream(byteStream)

core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala

Lines changed: 49 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -19,19 +19,19 @@ package org.apache.spark.deploy.history
1919

2020
import java.io.{File, FileNotFoundException, IOException}
2121
import java.util.{Date, ServiceLoader, UUID}
22-
import java.util.concurrent.{ExecutorService, TimeUnit}
22+
import java.util.concurrent.{ConcurrentHashMap, ExecutorService, Future, TimeUnit}
2323
import java.util.zip.{ZipEntry, ZipOutputStream}
2424

2525
import scala.collection.JavaConverters._
2626
import scala.collection.mutable
27+
import scala.concurrent.ExecutionException
2728
import scala.util.Try
2829
import scala.xml.Node
2930

3031
import com.fasterxml.jackson.annotation.JsonIgnore
3132
import com.google.common.io.ByteStreams
3233
import com.google.common.util.concurrent.MoreExecutors
33-
import org.apache.hadoop.fs.{FileStatus, Path}
34-
import org.apache.hadoop.fs.permission.FsAction
34+
import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
3535
import org.apache.hadoop.hdfs.DistributedFileSystem
3636
import org.apache.hadoop.hdfs.protocol.HdfsConstants
3737
import org.apache.hadoop.security.AccessControlException
@@ -111,7 +111,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
111111
"; groups with admin permissions" + HISTORY_UI_ADMIN_ACLS_GROUPS.toString)
112112

113113
private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf)
114-
private val fs = new Path(logDir).getFileSystem(hadoopConf)
114+
// Visible for testing
115+
private[history] val fs: FileSystem = new Path(logDir).getFileSystem(hadoopConf)
115116

116117
// Used by check event thread and clean log thread.
117118
// Scheduled thread pool size must be one, otherwise it will have concurrent issues about fs
@@ -155,6 +156,25 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
155156
new HistoryServerDiskManager(conf, path, listing, clock)
156157
}
157158

159+
private val blacklist = new ConcurrentHashMap[String, Long]
160+
161+
// Visible for testing
162+
private[history] def isBlacklisted(path: Path): Boolean = {
163+
blacklist.containsKey(path.getName)
164+
}
165+
166+
private def blacklist(path: Path): Unit = {
167+
blacklist.put(path.getName, clock.getTimeMillis())
168+
}
169+
170+
/**
171+
* Removes expired entries in the blacklist, according to the provided `expireTimeInSeconds`.
172+
*/
173+
private def clearBlacklist(expireTimeInSeconds: Long): Unit = {
174+
val expiredThreshold = clock.getTimeMillis() - expireTimeInSeconds * 1000
175+
blacklist.asScala.retain((_, creationTime) => creationTime >= expiredThreshold)
176+
}
177+
158178
private val activeUIs = new mutable.HashMap[(String, Option[String]), LoadedAppUI]()
159179

160180
/**
@@ -412,7 +432,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
412432
// reading a garbage file is safe, but we would log an error which can be scary to
413433
// the end-user.
414434
!entry.getPath().getName().startsWith(".") &&
415-
SparkHadoopUtil.get.checkAccessPermission(entry, FsAction.READ)
435+
!isBlacklisted(entry.getPath)
416436
}
417437
.filter { entry =>
418438
try {
@@ -446,32 +466,37 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
446466
logDebug(s"New/updated attempts found: ${updated.size} ${updated.map(_.getPath)}")
447467
}
448468

449-
val tasks = updated.map { entry =>
469+
val tasks = updated.flatMap { entry =>
450470
try {
451-
replayExecutor.submit(new Runnable {
471+
val task: Future[Unit] = replayExecutor.submit(new Runnable {
452472
override def run(): Unit = mergeApplicationListing(entry, newLastScanTime)
453-
})
473+
}, Unit)
474+
Some(task -> entry.getPath)
454475
} catch {
455476
// let the iteration over the updated entries break, since an exception on
456477
// replayExecutor.submit (..) indicates the ExecutorService is unable
457478
// to take any more submissions at this time
458479
case e: Exception =>
459480
logError(s"Exception while submitting event log for replay", e)
460-
null
481+
None
461482
}
462-
}.filter(_ != null)
483+
}
463484

464485
pendingReplayTasksCount.addAndGet(tasks.size)
465486

466487
// Wait for all tasks to finish. This makes sure that checkForLogs
467488
// is not scheduled again while some tasks are already running in
468489
// the replayExecutor.
469-
tasks.foreach { task =>
490+
tasks.foreach { case (task, path) =>
470491
try {
471492
task.get()
472493
} catch {
473494
case e: InterruptedException =>
474495
throw e
496+
case e: ExecutionException if e.getCause.isInstanceOf[AccessControlException] =>
497+
// We don't have read permissions on the log file
498+
logWarning(s"Unable to read log $path", e.getCause)
499+
blacklist(path)
475500
case e: Exception =>
476501
logError("Exception while merging application listings", e)
477502
} finally {
@@ -694,6 +719,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
694719
listing.delete(classOf[LogInfo], log.logPath)
695720
}
696721
}
722+
// Clean the blacklist from the expired entries.
723+
clearBlacklist(CLEAN_INTERVAL_S)
697724
}
698725

699726
/**
@@ -871,13 +898,17 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
871898
}
872899

873900
private def deleteLog(log: Path): Unit = {
874-
try {
875-
fs.delete(log, true)
876-
} catch {
877-
case _: AccessControlException =>
878-
logInfo(s"No permission to delete $log, ignoring.")
879-
case ioe: IOException =>
880-
logError(s"IOException in cleaning $log", ioe)
901+
if (isBlacklisted(log)) {
902+
logDebug(s"Skipping deleting $log as we don't have permissions on it.")
903+
} else {
904+
try {
905+
fs.delete(log, true)
906+
} catch {
907+
case _: AccessControlException =>
908+
logInfo(s"No permission to delete $log, ignoring.")
909+
case ioe: IOException =>
910+
logError(s"IOException in cleaning $log", ioe)
911+
}
881912
}
882913
}
883914

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

Lines changed: 0 additions & 97 deletions
This file was deleted.

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

Lines changed: 40 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -29,9 +29,11 @@ import scala.language.postfixOps
2929
import com.google.common.io.{ByteStreams, Files}
3030
import org.apache.hadoop.fs.{FileStatus, Path}
3131
import org.apache.hadoop.hdfs.DistributedFileSystem
32+
import org.apache.hadoop.security.AccessControlException
3233
import org.json4s.jackson.JsonMethods._
33-
import org.mockito.Matchers.any
34-
import org.mockito.Mockito.{doReturn, mock, spy, verify}
34+
import org.mockito.ArgumentMatcher
35+
import org.mockito.Matchers.{any, argThat}
36+
import org.mockito.Mockito.{doReturn, doThrow, mock, spy, verify, when}
3537
import org.scalatest.BeforeAndAfter
3638
import org.scalatest.Matchers
3739
import org.scalatest.concurrent.Eventually._
@@ -774,6 +776,42 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
774776
assert(new File(testDir.toURI).listFiles().size === validLogCount)
775777
}
776778

779+
test("SPARK-24948: blacklist files we don't have read permission on") {
780+
val clock = new ManualClock(1533132471)
781+
val provider = new FsHistoryProvider(createTestConf(), clock)
782+
val accessDenied = newLogFile("accessDenied", None, inProgress = false)
783+
writeFile(accessDenied, true, None,
784+
SparkListenerApplicationStart("accessDenied", Some("accessDenied"), 1L, "test", None))
785+
val accessGranted = newLogFile("accessGranted", None, inProgress = false)
786+
writeFile(accessGranted, true, None,
787+
SparkListenerApplicationStart("accessGranted", Some("accessGranted"), 1L, "test", None),
788+
SparkListenerApplicationEnd(5L))
789+
val mockedFs = spy(provider.fs)
790+
doThrow(new AccessControlException("Cannot read accessDenied file")).when(mockedFs).open(
791+
argThat(new ArgumentMatcher[Path]() {
792+
override def matches(path: Any): Boolean = {
793+
path.asInstanceOf[Path].getName.toLowerCase == "accessdenied"
794+
}
795+
}))
796+
val mockedProvider = spy(provider)
797+
when(mockedProvider.fs).thenReturn(mockedFs)
798+
updateAndCheck(mockedProvider) { list =>
799+
list.size should be(1)
800+
}
801+
writeFile(accessDenied, true, None,
802+
SparkListenerApplicationStart("accessDenied", Some("accessDenied"), 1L, "test", None),
803+
SparkListenerApplicationEnd(5L))
804+
// Doing 2 times in order to check the blacklist filter too
805+
updateAndCheck(mockedProvider) { list =>
806+
list.size should be(1)
807+
}
808+
val accessDeniedPath = new Path(accessDenied.getPath)
809+
assert(mockedProvider.isBlacklisted(accessDeniedPath))
810+
clock.advance(24 * 60 * 60 * 1000 + 1) // add a bit more than 1d
811+
mockedProvider.cleanLogs()
812+
assert(!mockedProvider.isBlacklisted(accessDeniedPath))
813+
}
814+
777815
/**
778816
* Asks the provider to check for logs and calls a function to perform checks on the updated
779817
* app list. Example:

0 commit comments

Comments
 (0)