Skip to content

Commit 187bb25

Browse files
committed
Formatting and renaming variables
1 parent 769336f commit 187bb25

File tree

2 files changed

+19
-15
lines changed

2 files changed

+19
-15
lines changed

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

Lines changed: 13 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.{FileSystem, Path}
2424
import org.json4s.jackson.JsonMethods._
2525

2626
import org.apache.spark.{Logging, SparkConf, SparkContext}
27+
import org.apache.spark.deploy.SparkHadoopUtil
2728
import org.apache.spark.io.CompressionCodec
2829
import org.apache.spark.util.{FileLogger, JsonProtocol}
2930

@@ -39,22 +40,25 @@ import org.apache.spark.util.{FileLogger, JsonProtocol}
3940
*/
4041
private[spark] class EventLoggingListener(
4142
appName: String,
42-
conf: SparkConf,
43-
hadoopConfiguration: Configuration)
43+
sparkConf: SparkConf,
44+
hadoopConf: Configuration)
4445
extends SparkListener with Logging {
4546

4647
import EventLoggingListener._
4748

48-
private val shouldCompress = conf.getBoolean("spark.eventLog.compress", false)
49-
private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", false)
50-
private val outputBufferSize = conf.getInt("spark.eventLog.buffer.kb", 100) * 1024
51-
private val logBaseDir = conf.get("spark.eventLog.dir", DEFAULT_LOG_DIR).stripSuffix("/")
49+
def this(appName: String, sparkConf: SparkConf) = {
50+
this(appName, sparkConf, SparkHadoopUtil.get.newConfiguration())
51+
}
52+
53+
private val shouldCompress = sparkConf.getBoolean("spark.eventLog.compress", false)
54+
private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false)
55+
private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024
56+
private val logBaseDir = sparkConf.get("spark.eventLog.dir", DEFAULT_LOG_DIR).stripSuffix("/")
5257
private val name = appName.replaceAll("[ :/]", "-").toLowerCase + "-" + System.currentTimeMillis
5358
val logDir = logBaseDir + "/" + name
5459

5560
private val logger =
56-
new FileLogger(logDir, conf, hadoopConfiguration, outputBufferSize, shouldCompress,
57-
shouldOverwrite)
61+
new FileLogger(logDir, sparkConf, hadoopConf, outputBufferSize, shouldCompress, shouldOverwrite)
5862

5963
/**
6064
* Begin logging events.
@@ -63,7 +67,7 @@ private[spark] class EventLoggingListener(
6367
def start() {
6468
logInfo("Logging events to %s".format(logDir))
6569
if (shouldCompress) {
66-
val codec = conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC)
70+
val codec = sparkConf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC)
6771
logger.newFile(COMPRESSION_CODEC_PREFIX + codec)
6872
}
6973
logger.newFile(SPARK_VERSION_PREFIX + SparkContext.SPARK_VERSION)

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

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.spark.util
1919

20-
import java.io.{FileOutputStream, BufferedOutputStream, PrintWriter, IOException}
20+
import java.io.{BufferedOutputStream, FileOutputStream, IOException, PrintWriter}
2121
import java.net.URI
2222
import java.text.SimpleDateFormat
2323
import java.util.Date
@@ -38,8 +38,8 @@ import org.apache.spark.io.CompressionCodec
3838
*/
3939
private[spark] class FileLogger(
4040
logDir: String,
41-
conf: SparkConf,
42-
hadoopConfiguration: Configuration,
41+
sparkConf: SparkConf,
42+
hadoopConf: Configuration,
4343
outputBufferSize: Int = 8 * 1024, // 8 KB
4444
compress: Boolean = false,
4545
overwrite: Boolean = true)
@@ -53,7 +53,7 @@ private[spark] class FileLogger(
5353
var fileIndex = 0
5454

5555
// Only used if compression is enabled
56-
private lazy val compressionCodec = CompressionCodec.createCodec(conf)
56+
private lazy val compressionCodec = CompressionCodec.createCodec(sparkConf)
5757

5858
// Only defined if the file system scheme is not local
5959
private var hadoopDataStream: Option[FSDataOutputStream] = None
@@ -87,8 +87,8 @@ private[spark] class FileLogger(
8787
private def createWriter(fileName: String): PrintWriter = {
8888
val logPath = logDir + "/" + fileName
8989
val uri = new URI(logPath)
90-
val defaultFs = FileSystem.getDefaultUri(hadoopConfiguration).getScheme
91-
val isDefaultLocal = (defaultFs == null || defaultFs == "file")
90+
val defaultFs = FileSystem.getDefaultUri(hadoopConf).getScheme
91+
val isDefaultLocal = defaultFs == null || defaultFs == "file"
9292

9393
/* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844).
9494
* Therefore, for local files, use FileOutputStream instead. */

0 commit comments

Comments
 (0)