-
Notifications
You must be signed in to change notification settings - Fork 29k
SPARK-1557 Set permissions on event log files/directories #538
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 4 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,6 +21,7 @@ import scala.collection.mutable | |
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.{FileSystem, Path} | ||
| import org.apache.hadoop.fs.permission.FsPermission | ||
| import org.json4s.jackson.JsonMethods._ | ||
|
|
||
| import org.apache.spark.{Logging, SparkConf, SparkContext} | ||
|
|
@@ -51,10 +52,12 @@ private[spark] class EventLoggingListener( | |
| private val logBaseDir = conf.get("spark.eventLog.dir", "/tmp/spark-events").stripSuffix("/") | ||
| private val name = appName.replaceAll("[ :/]", "-").toLowerCase + "-" + System.currentTimeMillis | ||
| val logDir = logBaseDir + "/" + name | ||
| val LOG_FILE_PERMISSIONS: FsPermission = | ||
| FsPermission.createImmutable(Integer.parseInt("770", 8).toShort: Short) | ||
|
|
||
| private val logger = | ||
| new FileLogger(logDir, conf, hadoopConfiguration, outputBufferSize, shouldCompress, | ||
| shouldOverwrite) | ||
| shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) | ||
|
|
||
| /** | ||
| * Begin logging events. | ||
|
|
@@ -64,10 +67,11 @@ private[spark] class EventLoggingListener( | |
| logInfo("Logging events to %s".format(logDir)) | ||
| if (shouldCompress) { | ||
| val codec = conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC) | ||
| logger.newFile(COMPRESSION_CODEC_PREFIX + codec) | ||
| logger.newFile(COMPRESSION_CODEC_PREFIX + codec, Some(LOG_FILE_PERMISSIONS)) | ||
| } | ||
| logger.newFile(SPARK_VERSION_PREFIX + SparkContext.SPARK_VERSION) | ||
| logger.newFile(LOG_PREFIX + logger.fileIndex) | ||
| logger.newFile(SPARK_VERSION_PREFIX + SparkContext.SPARK_VERSION, | ||
| Some(LOG_FILE_PERMISSIONS)) | ||
| logger.newFile(LOG_PREFIX + logger.fileIndex, Some(LOG_FILE_PERMISSIONS)) | ||
| } | ||
|
|
||
| /** Log the event as JSON. */ | ||
|
|
@@ -114,7 +118,7 @@ private[spark] class EventLoggingListener( | |
| * In addition, create an empty special file to indicate application completion. | ||
| */ | ||
| def stop() = { | ||
| logger.newFile(APPLICATION_COMPLETE) | ||
| logger.newFile(APPLICATION_COMPLETE, Some(LOG_FILE_PERMISSIONS)) | ||
|
||
| logger.stop() | ||
| } | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,6 +24,7 @@ import java.util.Date | |
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} | ||
| import org.apache.hadoop.fs.permission.FsPermission | ||
|
|
||
| import org.apache.spark.{Logging, SparkConf} | ||
| import org.apache.spark.io.CompressionCodec | ||
|
|
@@ -42,7 +43,8 @@ private[spark] class FileLogger( | |
| hadoopConfiguration: Configuration, | ||
| outputBufferSize: Int = 8 * 1024, // 8 KB | ||
| compress: Boolean = false, | ||
| overwrite: Boolean = true) | ||
| overwrite: Boolean = true, | ||
| dirPermissions: Option[FsPermission] = None) | ||
| extends Logging { | ||
|
|
||
| private val dateFormat = new ThreadLocal[SimpleDateFormat]() { | ||
|
|
@@ -79,16 +81,23 @@ private[spark] class FileLogger( | |
| if (!fileSystem.mkdirs(path)) { | ||
| throw new IOException("Error in creating log directory: %s".format(logDir)) | ||
| } | ||
| if (dirPermissions.isDefined) { | ||
| val fsStatus = fileSystem.getFileStatus(path) | ||
| if (fsStatus.getPermission().toShort() != dirPermissions.get.toShort()) { | ||
| fileSystem.setPermission(path, dirPermissions.get); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: semicolon, also in the line above, do |
||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Create a new writer for the file identified by the given path. | ||
| */ | ||
| private def createWriter(fileName: String): PrintWriter = { | ||
| private def createWriter(fileName: String, perms: Option[FsPermission] = None): PrintWriter = { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. same here
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The class says its a generic logging class. Left this to allow file permissions to be different per file.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sure |
||
| val logPath = logDir + "/" + fileName | ||
| val uri = new URI(logPath) | ||
| val defaultFs = FileSystem.getDefaultUri(hadoopConfiguration).getScheme | ||
| val isDefaultLocal = (defaultFs == null || defaultFs == "file") | ||
| val path = new Path(logPath) | ||
|
|
||
| /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). | ||
| * Therefore, for local files, use FileOutputStream instead. */ | ||
|
|
@@ -97,11 +106,11 @@ private[spark] class FileLogger( | |
| // Second parameter is whether to append | ||
| new FileOutputStream(uri.getPath, !overwrite) | ||
| } else { | ||
| val path = new Path(logPath) | ||
| hadoopDataStream = Some(fileSystem.create(path, overwrite)) | ||
| hadoopDataStream.get | ||
| } | ||
|
|
||
| perms.foreach {p => fileSystem.setPermission(path, p)} | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Here I would do |
||
| val bstream = new BufferedOutputStream(dstream, outputBufferSize) | ||
| val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream | ||
| new PrintWriter(cstream) | ||
|
|
@@ -150,15 +159,16 @@ private[spark] class FileLogger( | |
| /** | ||
| * Start a writer for a new file, closing the existing one if it exists. | ||
| * @param fileName Name of the new file, defaulting to the file index if not provided. | ||
| * @param perms Permissions to put on the new file. | ||
| */ | ||
| def newFile(fileName: String = "") { | ||
| def newFile(fileName: String = "", perms: Option[FsPermission] = None) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. same here (unless your intent was to have separate permissions on the file level granularity, which is also fine. Though this is not applicable to event logging)
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yep intent was to allow permissions to be set at file level if someone wanted. |
||
| fileIndex += 1 | ||
| writer.foreach(_.close()) | ||
| val name = fileName match { | ||
| case "" => fileIndex.toString | ||
| case _ => fileName | ||
| } | ||
| writer = Some(createWriter(name)) | ||
| writer = Some(createWriter(name, perms)) | ||
| } | ||
|
|
||
| /** | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No need to specify
: Short; it's pretty clear what it is withtoShortThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Also I think it's better if we move this to
object EventLoggingListener, alongside with other scary all caps variables.