Skip to content

Commit

Permalink
Updated based on comments
Browse files Browse the repository at this point in the history
  • Loading branch information
tgravescs committed Apr 25, 2014
1 parent 5a09709 commit 3ca9b79
Showing 1 changed file with 7 additions and 7 deletions.
14 changes: 7 additions & 7 deletions core/src/main/scala/org/apache/spark/util/FileLogger.scala
Original file line number Diff line number Diff line change
Expand Up @@ -24,10 +24,10 @@ 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
import org.apache.hadoop.fs.permission.FsPermission

/**
* A generic class for logging information to file.
Expand Down Expand Up @@ -62,12 +62,12 @@ private[spark] class FileLogger(

private var writer: Option[PrintWriter] = None

createLogDir(dirPermissions)
createLogDir()

/**
* Create a logging directory with the given path.
*/
private def createLogDir(dirPerms: Option[FsPermission]) {
private def createLogDir() {
val path = new Path(logDir)
if (fileSystem.exists(path)) {
if (overwrite) {
Expand All @@ -81,10 +81,10 @@ private[spark] class FileLogger(
if (!fileSystem.mkdirs(path)) {
throw new IOException("Error in creating log directory: %s".format(logDir))
}
if (dirPerms.isDefined) {
if (dirPermissions.isDefined) {
val fsStatus = fileSystem.getFileStatus(path)
if (fsStatus.getPermission().toShort() != dirPerms.get.toShort()) {
fileSystem.setPermission(path, dirPerms.get);
if (fsStatus.getPermission().toShort() != dirPermissions.get.toShort()) {
fileSystem.setPermission(path, dirPermissions.get);
}
}
}
Expand All @@ -110,7 +110,7 @@ private[spark] class FileLogger(
hadoopDataStream.get
}

if (perms.isDefined) fileSystem.setPermission(path, perms.get)
perms.foreach {p => fileSystem.setPermission(path, p)}
val bstream = new BufferedOutputStream(dstream, outputBufferSize)
val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream
new PrintWriter(cstream)
Expand Down

0 comments on commit 3ca9b79

Please sign in to comment.