Skip to content

Commit ad3106d

Browse files
tgravescspdeyhim
authored andcommitted
SPARK-1557 Set permissions on event log files/directories
This adds minimal setting of event log directory/files permissions. To have a secure environment the user must manually create the top level event log directory and set permissions up. We can add logic to do that automatically later if we want. Author: Thomas Graves <[email protected]> Closes apache#538 from tgravescs/SPARK-1557 and squashes the following commits: e471d8e [Thomas Graves] rework d8b6620 [Thomas Graves] update use of octal 3ca9b79 [Thomas Graves] Updated based on comments 5a09709 [Thomas Graves] add in missing import 3150ed6 [Thomas Graves] SPARK-1557 Set permissions on event log files/directories
1 parent 3d66228 commit ad3106d

File tree

3 files changed

+24
-6
lines changed

3 files changed

+24
-6
lines changed

core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ import scala.collection.mutable
2121

2222
import org.apache.hadoop.conf.Configuration
2323
import org.apache.hadoop.fs.{FileSystem, Path}
24+
import org.apache.hadoop.fs.permission.FsPermission
2425
import org.json4s.jackson.JsonMethods._
2526

2627
import org.apache.spark.{Logging, SparkConf, SparkContext}
@@ -54,7 +55,7 @@ private[spark] class EventLoggingListener(
5455

5556
private val logger =
5657
new FileLogger(logDir, conf, hadoopConfiguration, outputBufferSize, shouldCompress,
57-
shouldOverwrite)
58+
shouldOverwrite, Some(LOG_FILE_PERMISSIONS))
5859

5960
/**
6061
* Begin logging events.
@@ -124,6 +125,9 @@ private[spark] object EventLoggingListener extends Logging {
124125
val SPARK_VERSION_PREFIX = "SPARK_VERSION_"
125126
val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_"
126127
val APPLICATION_COMPLETE = "APPLICATION_COMPLETE"
128+
val LOG_FILE_PERMISSIONS: FsPermission =
129+
FsPermission.createImmutable(Integer.parseInt("770", 8).toShort)
130+
127131

128132
// A cache for compression codecs to avoid creating the same codec many times
129133
private val codecMap = new mutable.HashMap[String, CompressionCodec]

core/src/main/scala/org/apache/spark/util/FileLogger.scala

Lines changed: 17 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ import java.util.Date
2424

2525
import org.apache.hadoop.conf.Configuration
2626
import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
27+
import org.apache.hadoop.fs.permission.FsPermission
2728

2829
import org.apache.spark.{Logging, SparkConf}
2930
import org.apache.spark.io.CompressionCodec
@@ -42,7 +43,8 @@ private[spark] class FileLogger(
4243
hadoopConfiguration: Configuration,
4344
outputBufferSize: Int = 8 * 1024, // 8 KB
4445
compress: Boolean = false,
45-
overwrite: Boolean = true)
46+
overwrite: Boolean = true,
47+
dirPermissions: Option[FsPermission] = None)
4648
extends Logging {
4749

4850
private val dateFormat = new ThreadLocal[SimpleDateFormat]() {
@@ -79,16 +81,25 @@ private[spark] class FileLogger(
7981
if (!fileSystem.mkdirs(path)) {
8082
throw new IOException("Error in creating log directory: %s".format(logDir))
8183
}
84+
if (dirPermissions.isDefined) {
85+
val fsStatus = fileSystem.getFileStatus(path)
86+
if (fsStatus.getPermission().toShort() != dirPermissions.get.toShort) {
87+
fileSystem.setPermission(path, dirPermissions.get)
88+
}
89+
}
8290
}
8391

8492
/**
8593
* Create a new writer for the file identified by the given path.
94+
* If the permissions are not passed in, it will default to use the permissions
95+
* (dirpermissions) used when class was instantiated.
8696
*/
87-
private def createWriter(fileName: String): PrintWriter = {
97+
private def createWriter(fileName: String, perms: Option[FsPermission] = None): PrintWriter = {
8898
val logPath = logDir + "/" + fileName
8999
val uri = new URI(logPath)
90100
val defaultFs = FileSystem.getDefaultUri(hadoopConfiguration).getScheme
91101
val isDefaultLocal = (defaultFs == null || defaultFs == "file")
102+
val path = new Path(logPath)
92103

93104
/* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844).
94105
* Therefore, for local files, use FileOutputStream instead. */
@@ -97,11 +108,11 @@ private[spark] class FileLogger(
97108
// Second parameter is whether to append
98109
new FileOutputStream(uri.getPath, !overwrite)
99110
} else {
100-
val path = new Path(logPath)
101111
hadoopDataStream = Some(fileSystem.create(path, overwrite))
102112
hadoopDataStream.get
103113
}
104114

115+
perms.orElse(dirPermissions).foreach {p => fileSystem.setPermission(path, p)}
105116
val bstream = new BufferedOutputStream(dstream, outputBufferSize)
106117
val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream
107118
new PrintWriter(cstream)
@@ -152,15 +163,16 @@ private[spark] class FileLogger(
152163
/**
153164
* Start a writer for a new file, closing the existing one if it exists.
154165
* @param fileName Name of the new file, defaulting to the file index if not provided.
166+
* @param perms Permissions to put on the new file.
155167
*/
156-
def newFile(fileName: String = "") {
168+
def newFile(fileName: String = "", perms: Option[FsPermission] = None) {
157169
fileIndex += 1
158170
writer.foreach(_.close())
159171
val name = fileName match {
160172
case "" => fileIndex.toString
161173
case _ => fileName
162174
}
163-
writer = Some(createWriter(name))
175+
writer = Some(createWriter(name, perms))
164176
}
165177

166178
/**

docs/security.md

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -7,6 +7,8 @@ Spark currently supports authentication via a shared secret. Authentication can
77

88
The Spark UI can also be secured by using javax servlet filters. A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view acls to make sure they are authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls' control the behavior of the acls. Note that the person who started the application always has view access to the UI.
99

10+
If your applications are using event logging, the directory where the event logs go (`spark.eventLog.dir`) should be manually created and have the proper permissions set on it. If you want those log files secure, the permissions should be set to drwxrwxrwxt for that directory. The owner of the directory should be the super user who is running the history server and the group permissions should be restricted to super user group. This will allow all users to write to the directory but will prevent unprivileged users from removing or renaming a file unless they own the file or directory. The event log files will be created by Spark with permissions such that only the user and group have read and write access.
11+
1012
For Spark on Yarn deployments, configuring `spark.authenticate` to true will automatically handle generating and distributing the shared secret. Each application will use a unique shared secret. The Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. If an authentication filter is enabled, the acls controls can be used by control which users can via the Spark UI.
1113

1214
For other types of Spark deployments, the spark config `spark.authenticate.secret` should be configured on each of the nodes. This secret will be used by all the Master/Workers and applications. The UI can be secured using a javax servlet filter installed via `spark.ui.filters`. If an authentication filter is enabled, the acls controls can be used by control which users can via the Spark UI.

0 commit comments

Comments
 (0)