@@ -62,6 +62,13 @@ private[spark] class EventLoggingListener(
6262 private val testing = sparkConf.getBoolean(" spark.eventLog.testing" , false )
6363 private val outputBufferSize = sparkConf.getInt(" spark.eventLog.buffer.kb" , 100 ) * 1024
6464 private val fileSystem = Utils .getHadoopFileSystem(new URI (logBaseDir), hadoopConf)
65+ private val compressionCodec =
66+ if (shouldCompress) {
67+ Some (CompressionCodec .createCodec(sparkConf))
68+ } else {
69+ None
70+ }
71+ private val compressionCodecName = compressionCodec.map(_.getClass.getCanonicalName)
6572
6673 // Only defined if the file system scheme is not local
6774 private var hadoopDataStream : Option [FSDataOutputStream ] = None
@@ -80,7 +87,7 @@ private[spark] class EventLoggingListener(
8087 private [scheduler] val loggedEvents = new ArrayBuffer [JValue ]
8188
8289 // Visible for tests only.
83- private [scheduler] val logPath = getLogPath(logBaseDir, appId)
90+ private [scheduler] val logPath = getLogPath(logBaseDir, appId, compressionCodecName )
8491
8592 /**
8693 * Creates the log file in the configured log directory.
@@ -110,17 +117,12 @@ private[spark] class EventLoggingListener(
110117 hadoopDataStream = Some (fileSystem.create(path))
111118 hadoopDataStream.get
112119 }
113-
114- val compressionCodec =
115- if (shouldCompress) {
116- Some (CompressionCodec .createCodec(sparkConf))
117- } else {
118- None
119- }
120+ val cstream = compressionCodec.map(_.compressedOutputStream(dstream)).getOrElse(dstream)
121+ val bstream = new BufferedOutputStream (cstream, outputBufferSize)
120122
121123 fileSystem.setPermission(path, LOG_FILE_PERMISSIONS )
122- val logStream = initEventLog( new BufferedOutputStream (dstream, outputBufferSize),
123- compressionCodec)
124+
125+ val logStream = initEventLog(bstream, compressionCodec)
124126 writer = Some (new PrintWriter (logStream))
125127
126128 logInfo(" Logging events to %s" .format(logPath))
@@ -202,11 +204,11 @@ private[spark] object EventLoggingListener extends Logging {
202204 val IN_PROGRESS = " .inprogress"
203205 val DEFAULT_LOG_DIR = " /tmp/spark-events"
204206
205- private val LOG_FILE_PERMISSIONS = new FsPermission (Integer .parseInt(" 770" , 8 ).toShort)
207+ val EVENT_LOG_KEY = " EVENT_LOG"
208+ val SPARK_VERSION_KEY = " SPARK_VERSION"
209+ val COMPRESSION_CODEC_KEY = " COMPRESSION_CODEC"
206210
207- // Marker for the end of header data in a log file. After this marker, log data, potentially
208- // compressed, will be found.
209- private val HEADER_END_MARKER = " === LOG_HEADER_END ==="
211+ private val LOG_FILE_PERMISSIONS = new FsPermission (Integer .parseInt(" 770" , 8 ).toShort)
210212
211213 // To avoid corrupted files causing the heap to fill up. Value is arbitrary.
212214 private val MAX_HEADER_LINE_LENGTH = 4096
@@ -217,53 +219,60 @@ private[spark] object EventLoggingListener extends Logging {
217219 /**
218220 * Write metadata about the event log to the given stream.
219221 *
220- * The header is a serialized version of a map, except it does not use Java serialization to
221- * avoid incompatibilities between different JDKs. It writes one map entry per line, in
222- * "key=value" format.
223- *
224- * The very last entry in the header is the `HEADER_END_MARKER` marker, so that the parsing code
225- * can know when to stop.
222+ * The header is a single line of JSON in the beginning of the file. Note that this
223+ * assumes all metadata necessary to parse the log is also included in the file name.
224+ * The format needs to be kept in sync with the `openEventLog()` method below. Also, it
225+ * cannot change in new Spark versions without some other way of detecting the change.
226226 *
227- * The format needs to be kept in sync with the openEventLog() method below. Also, it cannot
228- * change in new Spark versions without some other way of detecting the change (like some
229- * metadata encoded in the file name).
230- *
231- * @param logStream Raw output stream to the even log file.
227+ * @param logStream Raw output stream to the event log file.
232228 * @param compressionCodec Optional compression codec to use.
233- * @return A stream where to write event log data. This may be a wrapper around the original
229+ * @return A stream to which event log data is written . This may be a wrapper around the original
234230 * stream (for example, when compression is enabled).
235231 */
236232 def initEventLog (
237233 logStream : OutputStream ,
238234 compressionCodec : Option [CompressionCodec ]): OutputStream = {
239- val meta = mutable.HashMap ((" version" -> SPARK_VERSION ))
235+ val metadata = new mutable.HashMap [String , String ]
236+ // Some of these metadata are already encoded in the file name
237+ // Here we include them again within the file itself for completeness
238+ metadata += (" Event" -> Utils .getFormattedClassName(SparkListenerMetadataIdentifier ))
239+ metadata += (SPARK_VERSION_KEY -> SPARK_VERSION )
240240 compressionCodec.foreach { codec =>
241- meta += (" compressionCodec " -> codec.getClass().getName() )
241+ metadata += (COMPRESSION_CODEC_KEY -> codec.getClass.getCanonicalName )
242242 }
243-
244- def write (entry : String ) = {
245- val bytes = entry.getBytes(Charsets .UTF_8 )
246- if (bytes.length > MAX_HEADER_LINE_LENGTH ) {
247- throw new IOException (s " Header entry too long: ${entry}" )
248- }
249- logStream.write(bytes, 0 , bytes.length)
243+ val metadataJson = compact(render(JsonProtocol .mapToJson(metadata)))
244+ val metadataBytes = (metadataJson + " \n " ).getBytes(Charsets .UTF_8 )
245+ if (metadataBytes.length > MAX_HEADER_LINE_LENGTH ) {
246+ throw new IOException (s " Event log metadata too long: $metadataJson" )
250247 }
251-
252- meta.foreach { case (k, v) => write(s " $k= $v\n " ) }
253- write(s " $HEADER_END_MARKER\n " )
254- compressionCodec.map(_.compressedOutputStream(logStream)).getOrElse(logStream)
248+ logStream.write(metadataBytes, 0 , metadataBytes.length)
249+ logStream
255250 }
256251
257252 /**
258253 * Return a file-system-safe path to the log file for the given application.
259254 *
255+ * Note that because we currently only create a single log file for each application,
256+ * we must encode all the information needed to parse this event log in the file name
257+ * instead of within the file itself. Otherwise, if the file is compressed, for instance,
258+ * we won't know which codec to use to decompress the metadata.
259+ *
260260 * @param logBaseDir Directory where the log file will be written.
261261 * @param appId A unique app ID.
262+ * @param compressionCodecName Name of the compression codec used to compress the contents
263+ * of the log, or None if compression is not enabled.
262264 * @return A path which consists of file-system-safe characters.
263265 */
264- def getLogPath (logBaseDir : String , appId : String ): String = {
265- val name = appId.replaceAll(" [ :/]" , " -" ).replaceAll(" [${}'\" ]" , " _" ).toLowerCase
266- Utils .resolveURI(logBaseDir) + " /" + name.stripSuffix(" /" )
266+ def getLogPath (
267+ logBaseDir : String ,
268+ appId : String ,
269+ compressionCodecName : Option [String ]): String = {
270+ val sanitizedAppId = appId.replaceAll(" [ :/]" , " -" ).replaceAll(" [${}'\" ]" , " _" ).toLowerCase
271+ // e.g. EVENT_LOG_app_123_SPARK_VERSION_1.3.1
272+ // e.g. EVENT_LOG_ {...} _COMPRESSION_CODEC_org.apache.spark.io.LZFCompressionCodec
273+ val logName = s " ${EVENT_LOG_KEY }_ ${sanitizedAppId}_ ${SPARK_VERSION_KEY }_ $SPARK_VERSION" +
274+ compressionCodecName.map { c => s " _ ${COMPRESSION_CODEC_KEY }_ $c" }.getOrElse(" " )
275+ Utils .resolveURI(logBaseDir).toString.stripSuffix(" /" ) + " /" + logName.stripSuffix(" /" )
267276 }
268277
269278 /**
@@ -279,51 +288,21 @@ private[spark] object EventLoggingListener extends Logging {
279288 }
280289
281290 val in = new BufferedInputStream (fs.open(log))
282- // Read a single line from the input stream without buffering.
283- // We cannot use BufferedReader because we must avoid reading
284- // beyond the end of the header, after which the content of the
285- // file may be compressed.
286- def readLine (): String = {
287- val bytes = new ByteArrayOutputStream ()
288- var next = in.read()
289- var count = 0
290- while (next != '\n ' ) {
291- if (next == - 1 ) {
292- throw new IOException (" Unexpected end of file." )
293- }
294- bytes.write(next)
295- count = count + 1
296- if (count > MAX_HEADER_LINE_LENGTH ) {
297- throw new IOException (" Maximum header line length exceeded." )
298- }
299- next = in.read()
300- }
301- new String (bytes.toByteArray(), Charsets .UTF_8 )
291+
292+ // Parse information from the log name
293+ val logName = log.getName
294+ val baseRegex = s " ${EVENT_LOG_KEY }_(.*)_ ${SPARK_VERSION_KEY }_(.*) " .r
295+ val compressionRegex = (baseRegex + s " _ ${COMPRESSION_CODEC_KEY }_(.*) " ).r
296+ val (sparkVersion, codecName) = logName match {
297+ case compressionRegex(_, version, _codecName) => (version, Some (_codecName))
298+ case baseRegex(_, version) => (version, None )
299+ case _ => throw new IllegalArgumentException (s " Malformed event log name: $logName" )
300+ }
301+ val codec = codecName.map { c =>
302+ codecMap.getOrElseUpdate(c, CompressionCodec .createCodec(new SparkConf , c))
302303 }
303304
304- // Parse the header metadata in the form of k=v pairs
305- // This assumes that every line before the header end marker follows this format
306305 try {
307- val meta = new mutable.HashMap [String , String ]()
308- var foundEndMarker = false
309- while (! foundEndMarker) {
310- readLine() match {
311- case HEADER_END_MARKER =>
312- foundEndMarker = true
313- case entry =>
314- val prop = entry.split(" =" , 2 )
315- if (prop.length != 2 ) {
316- throw new IllegalArgumentException (" Invalid metadata in log file." )
317- }
318- meta += (prop(0 ) -> prop(1 ))
319- }
320- }
321-
322- val sparkVersion = meta.get(" version" ).getOrElse(
323- throw new IllegalArgumentException (" Missing Spark version in log metadata." ))
324- val codec = meta.get(" compressionCodec" ).map { codecName =>
325- codecMap.getOrElseUpdate(codecName, CompressionCodec .createCodec(new SparkConf , codecName))
326- }
327306 (codec.map(_.compressedInputStream(in)).getOrElse(in), sparkVersion)
328307 } catch {
329308 case e : Exception =>
0 commit comments