@@ -23,14 +23,13 @@ import java.net.URI
2323import scala .collection .mutable
2424import scala .collection .mutable .ArrayBuffer
2525
26- import com .google .common .base .Charsets
2726import org .apache .hadoop .conf .Configuration
2827import org .apache .hadoop .fs .{FileSystem , FSDataOutputStream , Path }
2928import org .apache .hadoop .fs .permission .FsPermission
3029import org .json4s .JsonAST .JValue
3130import org .json4s .jackson .JsonMethods ._
3231
33- import org .apache .spark .{Logging , SparkConf , SPARK_VERSION }
32+ import org .apache .spark .{Logging , SparkConf }
3433import org .apache .spark .deploy .SparkHadoopUtil
3534import org .apache .spark .io .CompressionCodec
3635import org .apache .spark .util .{JsonProtocol , Utils }
@@ -68,7 +67,9 @@ private[spark] class EventLoggingListener(
6867 } else {
6968 None
7069 }
71- private val compressionCodecName = compressionCodec.map(_.getClass.getCanonicalName)
70+ private val compressionCodecName = compressionCodec.map { c =>
71+ CompressionCodec .getShortName(c.getClass.getName)
72+ }
7273
7374 // Only defined if the file system scheme is not local
7475 private var hadoopDataStream : Option [FSDataOutputStream ] = None
@@ -121,11 +122,8 @@ private[spark] class EventLoggingListener(
121122 try {
122123 val cstream = compressionCodec.map(_.compressedOutputStream(dstream)).getOrElse(dstream)
123124 val bstream = new BufferedOutputStream (cstream, outputBufferSize)
124-
125125 fileSystem.setPermission(path, LOG_FILE_PERMISSIONS )
126-
127- val logStream = initEventLog(bstream, compressionCodec)
128- writer = Some (new PrintWriter (logStream))
126+ writer = Some (new PrintWriter (bstream))
129127 logInfo(" Logging events to %s" .format(logPath))
130128 } catch {
131129 case e : Exception =>
@@ -214,56 +212,21 @@ private[spark] object EventLoggingListener extends Logging {
214212
215213 private val LOG_FILE_PERMISSIONS = new FsPermission (Integer .parseInt(" 770" , 8 ).toShort)
216214
217- // To avoid corrupted files causing the heap to fill up. Value is arbitrary.
218- private val MAX_HEADER_LINE_LENGTH = 4096
219-
220215 // A cache for compression codecs to avoid creating the same codec many times
221216 private val codecMap = new mutable.HashMap [String , CompressionCodec ]
222217
223- /**
224- * Write metadata about the event log to the given stream.
225- *
226- * The header is a single line of JSON in the beginning of the file. Note that this
227- * assumes all metadata necessary to parse the log is also included in the file name.
228- * The format needs to be kept in sync with the `openEventLog()` method below. Also, it
229- * cannot change in new Spark versions without some other way of detecting the change.
230- *
231- * @param logStream Raw output stream to the event log file.
232- * @param compressionCodec Optional compression codec to use.
233- * @return A stream to which event log data is written. This may be a wrapper around the original
234- * stream (for example, when compression is enabled).
235- */
236- def initEventLog (
237- logStream : OutputStream ,
238- compressionCodec : Option [CompressionCodec ]): OutputStream = {
239- val metadata = new mutable.HashMap [String , String ]
240- // Some of these metadata are already encoded in the file name
241- // Here we include them again within the file itself for completeness
242- metadata += (" Event" -> Utils .getFormattedClassName(SparkListenerMetadataIdentifier ))
243- metadata += (SPARK_VERSION_KEY -> SPARK_VERSION )
244- compressionCodec.foreach { codec =>
245- metadata += (COMPRESSION_CODEC_KEY -> codec.getClass.getCanonicalName)
246- }
247- val metadataJson = compact(render(JsonProtocol .mapToJson(metadata)))
248- val metadataBytes = (metadataJson + " \n " ).getBytes(Charsets .UTF_8 )
249- if (metadataBytes.length > MAX_HEADER_LINE_LENGTH ) {
250- throw new IOException (s " Event log metadata too long: $metadataJson" )
251- }
252- logStream.write(metadataBytes, 0 , metadataBytes.length)
253- logStream
254- }
255-
256218 /**
257219 * Return a file-system-safe path to the log file for the given application.
258220 *
259221 * Note that because we currently only create a single log file for each application,
260222 * we must encode all the information needed to parse this event log in the file name
261223 * instead of within the file itself. Otherwise, if the file is compressed, for instance,
262- * we won't know which codec to use to decompress the metadata.
224+ * we won't know which codec to use to decompress the metadata needed to open the file in
225+ * the first place.
263226 *
264227 * @param logBaseDir Directory where the log file will be written.
265228 * @param appId A unique app ID.
266- * @param compressionCodecName Name of the compression codec used to compress the contents
229+ * @param compressionCodecName Name to identify the codec used to compress the contents
267230 * of the log, or None if compression is not enabled.
268231 * @return A path which consists of file-system-safe characters.
269232 */
@@ -272,22 +235,19 @@ private[spark] object EventLoggingListener extends Logging {
272235 appId : String ,
273236 compressionCodecName : Option [String ] = None ): String = {
274237 val sanitizedAppId = appId.replaceAll(" [ :/]" , " -" ).replaceAll(" [${}'\" ]" , " _" ).toLowerCase
275- // e.g. EVENT_LOG_app_123_SPARK_VERSION_1.3.1
276- // e.g. EVENT_LOG_ {...} _COMPRESSION_CODEC_org.apache.spark.io.LZFCompressionCodec
277- val logName = s " ${sanitizedAppId} _ ${SPARK_VERSION_KEY }_ $SPARK_VERSION " +
278- compressionCodecName.map { c => s " _ ${ COMPRESSION_CODEC_KEY } _ $c " } .getOrElse(" " )
238+ // e.g. app_123, app_123_COMPRESSION_CODEC_lzf
239+ val logName = sanitizedAppId + compressionCodecName
240+ .map { c => s " _ ${COMPRESSION_CODEC_KEY }_ $c " }
241+ .getOrElse(" " )
279242 Utils .resolveURI(logBaseDir).toString.stripSuffix(" /" ) + " /" + logName
280243 }
281244
282245 /**
283246 * Opens an event log file and returns an input stream that contains the event data.
284247 *
285- * The first line of the returned input stream is a JSON header that describes the metadata
286- * of the event log.
287- *
288- * @return 2-tuple (event input stream, Spark version of event data)
248+ * @return input stream that holds one JSON serialized event per line
289249 */
290- def openEventLog (log : Path , fs : FileSystem ): ( InputStream , String ) = {
250+ def openEventLog (log : Path , fs : FileSystem ): InputStream = {
291251 // It's not clear whether FileSystem.open() throws FileNotFoundException or just plain
292252 // IOException when a file does not exist, so try our best to throw a proper exception.
293253 if (! fs.exists(log)) {
@@ -296,21 +256,19 @@ private[spark] object EventLoggingListener extends Logging {
296256
297257 val in = new BufferedInputStream (fs.open(log))
298258
299- // Parse information from the log name
259+ // Parse compression codec from the log name
300260 val logName = log.getName
301- val baseRegex = s " (.*)_ ${SPARK_VERSION_KEY }_(.*) " .r
302- val compressionRegex = (baseRegex + s " _ ${COMPRESSION_CODEC_KEY }_(.*) " ).r
303- val (sparkVersion, codecName) = logName match {
304- case compressionRegex(_, version, _codecName) => (version, Some (_codecName))
305- case baseRegex(_, version) => (version, None )
306- case _ => throw new IllegalArgumentException (s " Malformed event log name: $logName" )
261+ val compressionRegex = s " .*_ ${COMPRESSION_CODEC_KEY }_(.*) " .r
262+ val codecName : Option [String ] = logName match {
263+ case compressionRegex(_codecName) => Some (_codecName)
264+ case _ => None
307265 }
308266 val codec = codecName.map { c =>
309267 codecMap.getOrElseUpdate(c, CompressionCodec .createCodec(new SparkConf , c))
310268 }
311269
312270 try {
313- ( codec.map(_.compressedInputStream(in)).getOrElse(in), sparkVersion )
271+ codec.map(_.compressedInputStream(in)).getOrElse(in)
314272 } catch {
315273 case e : Exception =>
316274 in.close()
0 commit comments