Skip to content

Commit 4b203f1

Browse files
author
zuotingbing
committed
[SPARK-20338]Spaces in spark.eventLog.dir are not correctly handled
1 parent 7536e28 commit 4b203f1

File tree

6 files changed

+20
-30
lines changed

6 files changed

+20
-30
lines changed

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 3 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -193,7 +193,7 @@ class SparkContext(config: SparkConf) extends Logging {
193193
* ------------------------------------------------------------------------------------- */
194194

195195
private var _conf: SparkConf = _
196-
private var _eventLogDir: Option[URI] = None
196+
private var _eventLogDir: Option[String] = None
197197
private var _eventLogCodec: Option[String] = None
198198
private var _env: SparkEnv = _
199199
private var _jobProgressListener: JobProgressListener = _
@@ -236,7 +236,7 @@ class SparkContext(config: SparkConf) extends Logging {
236236
def appName: String = _conf.get("spark.app.name")
237237

238238
private[spark] def isEventLogEnabled: Boolean = _conf.getBoolean("spark.eventLog.enabled", false)
239-
private[spark] def eventLogDir: Option[URI] = _eventLogDir
239+
private[spark] def eventLogDir: Option[String] = _eventLogDir
240240
private[spark] def eventLogCodec: Option[String] = _eventLogCodec
241241

242242
def isLocal: Boolean = Utils.isLocalMaster(_conf)
@@ -405,9 +405,7 @@ class SparkContext(config: SparkConf) extends Logging {
405405

406406
_eventLogDir =
407407
if (isEventLogEnabled) {
408-
val unresolvedDir = conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR)
409-
.stripSuffix("/")
410-
Some(Utils.resolveURI(unresolvedDir))
408+
Some(conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR).stripSuffix("/"))
411409
} else {
412410
None
413411
}

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

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,15 +17,13 @@
1717

1818
package org.apache.spark.deploy
1919

20-
import java.net.URI
21-
2220
private[spark] case class ApplicationDescription(
2321
name: String,
2422
maxCores: Option[Int],
2523
memoryPerExecutorMB: Int,
2624
command: Command,
2725
appUiUrl: String,
28-
eventLogDir: Option[URI] = None,
26+
eventLogDir: Option[String] = None,
2927
// short name of compression codec used when writing event logs, if any (e.g. lzf)
3028
eventLogCodec: Option[String] = None,
3129
coresPerExecutor: Option[Int] = None,

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

Lines changed: 6 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@
1818
package org.apache.spark.scheduler
1919

2020
import java.io._
21-
import java.net.URI
2221
import java.nio.charset.StandardCharsets
2322
import java.util.Locale
2423

@@ -50,22 +49,22 @@ import org.apache.spark.util.{JsonProtocol, Utils}
5049
private[spark] class EventLoggingListener(
5150
appId: String,
5251
appAttemptId : Option[String],
53-
logBaseDir: URI,
52+
logBaseDir: String,
5453
sparkConf: SparkConf,
5554
hadoopConf: Configuration)
5655
extends SparkListener with Logging {
5756

5857
import EventLoggingListener._
5958

60-
def this(appId: String, appAttemptId : Option[String], logBaseDir: URI, sparkConf: SparkConf) =
59+
def this(appId: String, appAttemptId : Option[String], logBaseDir: String, sparkConf: SparkConf) =
6160
this(appId, appAttemptId, logBaseDir, sparkConf,
6261
SparkHadoopUtil.get.newConfiguration(sparkConf))
6362

6463
private val shouldCompress = sparkConf.getBoolean("spark.eventLog.compress", false)
6564
private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false)
6665
private val testing = sparkConf.getBoolean("spark.eventLog.testing", false)
6766
private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024
68-
private val fileSystem = Utils.getHadoopFileSystem(logBaseDir, hadoopConf)
67+
private val fileSystem = new Path(logBaseDir).getFileSystem(hadoopConf)
6968
private val compressionCodec =
7069
if (shouldCompress) {
7170
Some(CompressionCodec.createCodec(sparkConf))
@@ -96,8 +95,8 @@ private[spark] class EventLoggingListener(
9695
}
9796

9897
val workingPath = logPath + IN_PROGRESS
99-
val uri = new URI(workingPath)
10098
val path = new Path(workingPath)
99+
val uri = path.toUri
101100
val defaultFs = FileSystem.getDefaultUri(hadoopConf).getScheme
102101
val isDefaultLocal = defaultFs == null || defaultFs == "file"
103102

@@ -303,11 +302,11 @@ private[spark] object EventLoggingListener extends Logging {
303302
* @return A path which consists of file-system-safe characters.
304303
*/
305304
def getLogPath(
306-
logBaseDir: URI,
305+
logBaseDir: String,
307306
appId: String,
308307
appAttemptId: Option[String],
309308
compressionCodecName: Option[String] = None): String = {
310-
val base = logBaseDir.toString.stripSuffix("/") + "/" + sanitize(appId)
309+
val base = logBaseDir.stripSuffix("/") + "/" + sanitize(appId)
311310
val codec = compressionCodecName.map("." + _).getOrElse("")
312311
if (appAttemptId.isDefined) {
313312
base + "_" + sanitize(appAttemptId.get) + codec

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

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -61,8 +61,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
6161
inProgress: Boolean,
6262
codec: Option[String] = None): File = {
6363
val ip = if (inProgress) EventLoggingListener.IN_PROGRESS else ""
64-
val logUri = EventLoggingListener.getLogPath(testDir.toURI, appId, appAttemptId)
65-
val logPath = new URI(logUri).getPath + ip
64+
val logPath = EventLoggingListener.getLogPath(testDir.getAbsolutePath, appId, appAttemptId) + ip
6665
new File(logPath)
6766
}
6867

core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala

Lines changed: 8 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -50,7 +50,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
5050
private var testDirPath: Path = _
5151

5252
before {
53-
testDir = Utils.createTempDir()
53+
testDir = Utils.createTempDir(namePrefix = s"event log")
5454
testDir.deleteOnExit()
5555
testDirPath = new Path(testDir.getAbsolutePath())
5656
}
@@ -62,7 +62,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
6262
test("Verify log file exist") {
6363
// Verify logging directory exists
6464
val conf = getLoggingConf(testDirPath)
65-
val eventLogger = new EventLoggingListener("test", None, testDirPath.toUri(), conf)
65+
val eventLogger = new EventLoggingListener("test", None, testDirPath.toString, conf)
6666
eventLogger.start()
6767

6868
val logPath = new Path(eventLogger.logPath + EventLoggingListener.IN_PROGRESS)
@@ -100,16 +100,15 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
100100
val secretPassword = "secret_password"
101101
val conf = getLoggingConf(testDirPath, None)
102102
.set(key, secretPassword)
103-
val eventLogger = new EventLoggingListener("test", None, testDirPath.toUri(), conf)
103+
val eventLogger = new EventLoggingListener("test", None, testDirPath.toString, conf)
104104
val envDetails = SparkEnv.environmentDetails(conf, "FIFO", Seq.empty, Seq.empty)
105105
val event = SparkListenerEnvironmentUpdate(envDetails)
106106
val redactedProps = eventLogger.redactEvent(event).environmentDetails("Spark Properties").toMap
107107
assert(redactedProps(key) == "*********(redacted)")
108108
}
109109

110110
test("Log overwriting") {
111-
val logUri = EventLoggingListener.getLogPath(testDir.toURI, "test", None)
112-
val logPath = new URI(logUri).getPath
111+
val logPath = EventLoggingListener.getLogPath(testDir.toString, "test", None)
113112
// Create file before writing the event log
114113
new FileOutputStream(new File(logPath)).close()
115114
// Expected IOException, since we haven't enabled log overwrite.
@@ -119,7 +118,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
119118
}
120119

121120
test("Event log name") {
122-
val baseDirUri = Utils.resolveURI("/base-dir")
121+
val baseDirUri = "/base-dir"
123122
// without compression
124123
assert(s"${baseDirUri.toString}/app1" === EventLoggingListener.getLogPath(
125124
baseDirUri, "app1", None))
@@ -154,7 +153,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
154153
val conf = getLoggingConf(testDirPath, compressionCodec)
155154
extraConf.foreach { case (k, v) => conf.set(k, v) }
156155
val logName = compressionCodec.map("test-" + _).getOrElse("test")
157-
val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
156+
val eventLogger = new EventLoggingListener(logName, None, testDirPath.toString, conf)
158157
val listenerBus = new LiveListenerBus(sc)
159158
val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None,
160159
125L, "Mickey", None)
@@ -190,15 +189,12 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
190189
* This runs a simple Spark job and asserts that the expected events are logged when expected.
191190
*/
192191
private def testApplicationEventLogging(compressionCodec: Option[String] = None) {
193-
// Set defaultFS to something that would cause an exception, to make sure we don't run
194-
// into SPARK-6688.
195192
val conf = getLoggingConf(testDirPath, compressionCodec)
196-
.set("spark.hadoop.fs.defaultFS", "unsupported://example.com")
197193
sc = new SparkContext("local-cluster[2,2,1024]", "test", conf)
198194
assert(sc.eventLogger.isDefined)
199195
val eventLogger = sc.eventLogger.get
200196
val eventLogPath = eventLogger.logPath
201-
val expectedLogDir = testDir.toURI()
197+
val expectedLogDir = testDir.getAbsolutePath
202198
assert(eventLogPath === EventLoggingListener.getLogPath(
203199
expectedLogDir, sc.applicationId, None, compressionCodec.map(CompressionCodec.getShortName)))
204200

@@ -290,7 +286,7 @@ object EventLoggingListenerSuite {
290286
val conf = new SparkConf
291287
conf.set("spark.eventLog.enabled", "true")
292288
conf.set("spark.eventLog.testing", "true")
293-
conf.set("spark.eventLog.dir", logDir.toUri.toString)
289+
conf.set("spark.eventLog.dir", logDir.toString)
294290
compressionCodec.foreach { codec =>
295291
conf.set("spark.eventLog.compress", "true")
296292
conf.set("spark.io.compression.codec", codec)

core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -151,7 +151,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp
151151
* log the events.
152152
*/
153153
private class EventMonster(conf: SparkConf)
154-
extends EventLoggingListener("test", None, new URI("testdir"), conf) {
154+
extends EventLoggingListener("test", None, "test dir", conf) {
155155

156156
override def start() { }
157157

0 commit comments

Comments
 (0)