From 6f3dd79856863e195923392afd43710e593d3872 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 26 Aug 2019 22:09:01 +0900 Subject: [PATCH 01/15] [SPARK-28869][CORE] Roll over event log files * Reflect first round of review comments from vanzin * Rename the term "sequence (number)" to "index" --- .../deploy/history/EventLogFileReaders.scala | 248 +++++++++++ .../deploy/history/EventLogFileWriters.scala | 421 ++++++++++++++++++ .../deploy/history/FsHistoryProvider.scala | 184 ++++---- .../spark/internal/config/package.scala | 11 + .../scheduler/EventLoggingListener.scala | 222 ++------- .../spark/deploy/SparkSubmitSuite.scala | 4 +- .../history/EventLogFileReadersSuite.scala | 345 ++++++++++++++ .../history/EventLogFileWritersSuite.scala | 400 +++++++++++++++++ .../deploy/history/EventLogTestHelper.scala | 45 ++ .../history/FsHistoryProviderSuite.scala | 50 ++- .../scheduler/EventLoggingListenerSuite.scala | 108 +---- .../spark/scheduler/ReplayListenerSuite.scala | 17 +- docs/configuration.md | 15 + 13 files changed, 1665 insertions(+), 405 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala new file mode 100644 index 000000000000..7e46d2a425cd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala @@ -0,0 +1,248 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import java.io.{BufferedInputStream, InputStream} +import java.util.zip.{ZipEntry, ZipOutputStream} + +import scala.collection.mutable + +import com.google.common.io.ByteStreams +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.hdfs.DFSInputStream + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.history.EventLogFileWriter.codecName +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.Utils + +/** The base class of reader which will read the information of event log file(s). */ +abstract class EventLogFileReader( + protected val fileSystem: FileSystem, + val rootPath: Path) { + + protected def fileSizeForDFS(path: Path): Option[Long] = { + Utils.tryWithResource(fileSystem.open(path)) { in => + in.getWrappedStream match { + case dfsIn: DFSInputStream => Some(dfsIn.getFileLength) + case _ => None + } + } + } + + protected def addFileAsZipEntry( + zipStream: ZipOutputStream, + path: Path, + entryName: String): Unit = { + Utils.tryWithResource(fileSystem.open(path, 1 * 1024 * 1024)) { inputStream => + zipStream.putNextEntry(new ZipEntry(entryName)) + ByteStreams.copy(inputStream, zipStream) + zipStream.closeEntry() + } + } + + /** Returns the last index of event log files. None for single event log file. */ + def lastIndex: Option[Long] + + /** + * Returns the size of file for the last index of event log files. Returns its size for + * single event log file. + */ + def fileSizeForLastIndex: Long + + /** Returns whether the application is completed. */ + def completed: Boolean + + /** + * Returns the size of file for the last index of event log files, only when + * underlying input stream is DFSInputStream. Otherwise returns None. + */ + def fileSizeForLastIndexForDFS: Option[Long] + + /** Returns the modification time for the last sequence number of event log files. */ + def modificationTime: Long + + /** + * This method compresses the files passed in, and writes the compressed data out into the + * ZipOutputStream passed in. Each file is written as a new ZipEntry with its name being + * the name of the file being compressed. + */ + def zipEventLogFiles(zipStream: ZipOutputStream): Unit + + /** Returns all available event log files. */ + def listEventLogFiles: Seq[FileStatus] + + /** Returns the short compression name if being used. None if it's uncompressed. */ + def compressionCodec: Option[String] + + /** Returns the size of all event log files. */ + def totalSize: Long +} + +object EventLogFileReader { + // A cache for compression codecs to avoid creating the same codec many times + private val codecMap = mutable.HashMap.empty[String, CompressionCodec] + + def apply( + fs: FileSystem, + path: Path, + lastSequenceNumber: Option[Long]): EventLogFileReader = { + lastSequenceNumber match { + case Some(_) => new RollingEventLogFilesFileReader(fs, path) + case None => new SingleFileEventLogFileReader(fs, path) + } + } + + def apply(fs: FileSystem, path: Path): Option[EventLogFileReader] = { + apply(fs, fs.getFileStatus(path)) + } + + def apply(fs: FileSystem, status: FileStatus): Option[EventLogFileReader] = { + if (isSingleEventLog(status)) { + Some(new SingleFileEventLogFileReader(fs, status.getPath)) + } else if (isRollingEventLogs(status)) { + Some(new RollingEventLogFilesFileReader(fs, status.getPath)) + } else { + None + } + } + + /** + * Opens an event log file and returns an input stream that contains the event data. + * + * @return input stream that holds one JSON record per line. + */ + def openEventLog(log: Path, fs: FileSystem): InputStream = { + val in = new BufferedInputStream(fs.open(log)) + try { + val codec = codecName(log).map { c => + codecMap.getOrElseUpdate(c, CompressionCodec.createCodec(new SparkConf, c)) + } + codec.map(_.compressedContinuousInputStream(in)).getOrElse(in) + } catch { + case e: Throwable => + in.close() + throw e + } + } + + private def isSingleEventLog(status: FileStatus): Boolean = { + !status.isDirectory && + // FsHistoryProvider used to generate a hidden file which can't be read. Accidentally + // reading a garbage file is safe, but we would log an error which can be scary to + // the end-user. + !status.getPath.getName.startsWith(".") + } + + private def isRollingEventLogs(status: FileStatus): Boolean = { + status.isDirectory && RollingEventLogFilesWriter.isEventLogDir(status) + } +} + +/** The reader which will read the information of single event log file. */ +class SingleFileEventLogFileReader( + fs: FileSystem, + path: Path) extends EventLogFileReader(fs, path) { + // TODO: get stats with constructor and only call if it's needed? + private lazy val status = fileSystem.getFileStatus(rootPath) + + override def lastIndex: Option[Long] = None + + override def fileSizeForLastIndex: Long = status.getLen + + override def completed: Boolean = !rootPath.getName.endsWith(EventLogFileWriter.IN_PROGRESS) + + override def fileSizeForLastIndexForDFS: Option[Long] = { + if (completed) { + Some(fileSizeForLastIndex) + } else { + fileSizeForDFS(rootPath) + } + } + + override def modificationTime: Long = status.getModificationTime + + override def zipEventLogFiles(zipStream: ZipOutputStream): Unit = + addFileAsZipEntry(zipStream, rootPath, rootPath.getName) + + override def listEventLogFiles: Seq[FileStatus] = Seq(status) + + override def compressionCodec: Option[String] = EventLogFileWriter.codecName(rootPath) + + override def totalSize: Long = fileSizeForLastIndex +} + +/** The reader which will read the information of rolled multiple event log files. */ +class RollingEventLogFilesFileReader( + fs: FileSystem, + path: Path) extends EventLogFileReader(fs, path) { + import RollingEventLogFilesWriter._ + + private lazy val files: Seq[FileStatus] = { + val ret = fs.listStatus(rootPath).toSeq + require(ret.exists(isEventLogFile), "Log directory must contain at least one event log file!") + require(ret.exists(isAppStatusFile), "Log directory must contain an appstatus file!") + ret + } + + override def lastIndex: Option[Long] = { + val maxSeq = files.filter(isEventLogFile) + .map { stats => getSequence(stats.getPath.getName) } + .max + Some(maxSeq) + } + + override def fileSizeForLastIndex: Long = lastEventLogFile.getLen + + override def completed: Boolean = { + val appStatsFile = files.find(isAppStatusFile) + require(appStatsFile.isDefined) + appStatsFile.exists(!_.getPath.getName.endsWith(EventLogFileWriter.IN_PROGRESS)) + } + + override def fileSizeForLastIndexForDFS: Option[Long] = { + if (completed) { + Some(fileSizeForLastIndex) + } else { + fileSizeForDFS(lastEventLogFile.getPath) + } + } + + override def modificationTime: Long = lastEventLogFile.getModificationTime + + override def zipEventLogFiles(zipStream: ZipOutputStream): Unit = { + val dirEntryName = rootPath.getName + "/" + zipStream.putNextEntry(new ZipEntry(dirEntryName)) + files.foreach { file => + addFileAsZipEntry(zipStream, file.getPath, dirEntryName + file.getPath.getName) + } + } + + override def listEventLogFiles: Seq[FileStatus] = eventLogFiles + + override def compressionCodec: Option[String] = EventLogFileWriter.codecName( + eventLogFiles.head.getPath) + + override def totalSize: Long = eventLogFiles.map(_.getLen).sum + + private def eventLogFiles: Seq[FileStatus] = { + files.filter(isEventLogFile).sortBy { stats => getSequence(stats.getPath.getName) } + } + + private def lastEventLogFile: FileStatus = eventLogFiles.reverse.head +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala new file mode 100644 index 000000000000..2d9cd0ce208a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -0,0 +1,421 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import java.io._ +import java.net.URI +import java.nio.charset.StandardCharsets + +import org.apache.commons.compress.utils.CountingOutputStream +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, FileSystem, FSDataOutputStream, Path} +import org.apache.hadoop.fs.permission.FsPermission + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.Utils + +/** + * The base class of writer which will write event logs into file. + * + * The following configurable parameters are available to tune the behavior of writing: + * spark.eventLog.compress - Whether to compress logged events + * spark.eventLog.compression.codec - The codec to compress logged events + * spark.eventLog.overwrite - Whether to overwrite any existing files + * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams + * + * Note that descendant classes can maintain its own parameters: refer the javadoc of each class + * for more details. + * + * NOTE: CountingOutputStream being returned by "initLogFile" counts "non-compressed" bytes. + */ +abstract class EventLogFileWriter( + appId: String, + appAttemptId : Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration) extends Logging { + + protected val shouldCompress = sparkConf.get(EVENT_LOG_COMPRESS) + protected val shouldOverwrite = sparkConf.get(EVENT_LOG_OVERWRITE) + protected val shouldAllowECLogs = sparkConf.get(EVENT_LOG_ALLOW_EC) + protected val outputBufferSize = sparkConf.get(EVENT_LOG_OUTPUT_BUFFER_SIZE).toInt + protected val fileSystem = Utils.getHadoopFileSystem(logBaseDir, hadoopConf) + protected val compressionCodec = + if (shouldCompress) { + Some(CompressionCodec.createCodec(sparkConf, sparkConf.get(EVENT_LOG_COMPRESSION_CODEC))) + } else { + None + } + + private[history] val compressionCodecName = compressionCodec.map { c => + CompressionCodec.getShortName(c.getClass.getName) + } + + protected def requireLogBaseDirAsDirectory(): Unit = { + if (!fileSystem.getFileStatus(new Path(logBaseDir)).isDirectory) { + throw new IllegalArgumentException(s"Log directory $logBaseDir is not a directory.") + } + } + + protected def initLogFile(path: Path): (Option[FSDataOutputStream], OutputStream) = { + if (shouldOverwrite && fileSystem.delete(path, true)) { + logWarning(s"Event log $path already exists. Overwriting...") + } + + val defaultFs = FileSystem.getDefaultUri(hadoopConf).getScheme + val isDefaultLocal = defaultFs == null || defaultFs == "file" + val uri = path.toUri + + var hadoopDataStream: Option[FSDataOutputStream] = None + /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). + * Therefore, for local files, use FileOutputStream instead. */ + val dstream = + if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { + new FileOutputStream(uri.getPath) + } else { + hadoopDataStream = Some( + SparkHadoopUtil.createFile(fileSystem, path, sparkConf.get(EVENT_LOG_ALLOW_EC))) + hadoopDataStream.get + } + + try { + val cstream = compressionCodec.map(_.compressedContinuousOutputStream(dstream)) + .getOrElse(dstream) + val bstream = new BufferedOutputStream(cstream, outputBufferSize) + fileSystem.setPermission(path, EventLogFileWriter.LOG_FILE_PERMISSIONS) + logInfo(s"Logging events to $path") + (hadoopDataStream, bstream) + } catch { + case e: Exception => + dstream.close() + throw e + } + } + + protected def renameFile(src: Path, dest: Path, overwrite: Boolean): Unit = { + if (fileSystem.exists(dest)) { + if (overwrite) { + logWarning(s"Event log $dest already exists. Overwriting...") + if (!fileSystem.delete(dest, true)) { + logWarning(s"Error deleting $dest") + } + } else { + throw new IOException(s"Target log file already exists ($dest)") + } + } + fileSystem.rename(src, dest) + // touch file to ensure modtime is current across those filesystems where rename() + // does not set it, -and which support setTimes(); it's a no-op on most object stores + try { + fileSystem.setTimes(dest, System.currentTimeMillis(), -1) + } catch { + case e: Exception => logDebug(s"failed to set time of $dest", e) + } + } + + /** initialize writer for event logging */ + def start(): Unit + + /** writes JSON format of event to file */ + def writeEvent(eventJson: String, flushLogger: Boolean = false): Unit + + /** stops writer - indicating the application has been completed */ + def stop(): Unit + + /** returns representative path of log */ + def logPath: String +} + +object EventLogFileWriter { + // Suffix applied to the names of files still being written by applications. + val IN_PROGRESS = ".inprogress" + + val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) + + def apply( + appId: String, + appAttemptId: Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration): EventLogFileWriter = { + if (sparkConf.get(EVENT_LOG_ENABLE_ROLLING)) { + new RollingEventLogFilesWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) + } else { + new SingleEventLogFileWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) + } + } + + def nameForAppAndAttempt(appId: String, appAttemptId: Option[String]): String = { + val base = Utils.sanitizeDirName(appId) + if (appAttemptId.isDefined) { + base + "_" + Utils.sanitizeDirName(appAttemptId.get) + } else { + base + } + } + + def codecName(log: Path): Option[String] = { + // Compression codec is encoded as an extension, e.g. app_123.lzf + // Since we sanitize the app ID to not include periods, it is safe to split on it + val logName = log.getName.stripSuffix(IN_PROGRESS) + logName.split("\\.").tail.lastOption + } +} + +/** + * The writer to write event logs into single file. + */ +class SingleEventLogFileWriter( + appId: String, + appAttemptId : Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration) + extends EventLogFileWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) { + + override val logPath: String = SingleEventLogFileWriter.getLogPath(logBaseDir, appId, + appAttemptId, compressionCodecName) + + private val inProgressPath = logPath + EventLogFileWriter.IN_PROGRESS + + // Only defined if the file system scheme is not local + private var hadoopDataStream: Option[FSDataOutputStream] = None + + private var writer: Option[PrintWriter] = None + + override def start(): Unit = { + requireLogBaseDirAsDirectory() + + val (hadoopStream, outputStream) = initLogFile(new Path(inProgressPath)) + hadoopDataStream = hadoopStream + writer = Some(new PrintWriter(new OutputStreamWriter(outputStream, StandardCharsets.UTF_8))) + } + + override def writeEvent(eventJson: String, flushLogger: Boolean = false): Unit = { + // scalastyle:off println + writer.foreach(_.println(eventJson)) + // scalastyle:on println + if (flushLogger) { + writer.foreach(_.flush()) + hadoopDataStream.foreach(_.hflush()) + } + } + + /** + * Stop logging events. The event log file will be renamed so that it loses the + * ".inprogress" suffix. + */ + override def stop(): Unit = { + writer.foreach(_.close()) + renameFile(new Path(inProgressPath), new Path(logPath), shouldOverwrite) + } +} + +object SingleEventLogFileWriter { + /** + * Return a file-system-safe path to the log file for the given application. + * + * Note that because we currently only create a single log file for each application, + * we must encode all the information needed to parse this event log in the file name + * instead of within the file itself. Otherwise, if the file is compressed, for instance, + * we won't know which codec to use to decompress the metadata needed to open the file in + * the first place. + * + * The log file name will identify the compression codec used for the contents, if any. + * For example, app_123 for an uncompressed log, app_123.lzf for an LZF-compressed log. + * + * @param logBaseDir Directory where the log file will be written. + * @param appId A unique app ID. + * @param appAttemptId A unique attempt id of appId. May be the empty string. + * @param compressionCodecName Name to identify the codec used to compress the contents + * of the log, or None if compression is not enabled. + * @return A path which consists of file-system-safe characters. + */ + def getLogPath( + logBaseDir: URI, + appId: String, + appAttemptId: Option[String], + compressionCodecName: Option[String] = None): String = { + val codec = compressionCodecName.map("." + _).getOrElse("") + new Path(logBaseDir).toString.stripSuffix("/") + "/" + + EventLogFileWriter.nameForAppAndAttempt(appId, appAttemptId) + codec + } +} + +/** + * The writer to write event logs into multiple log files, rolled over via configured size. + * + * The class creates one directory per application, and stores event log files as well as + * metadata files. The name of directory and files in the directory would follow: + * + * - The name of directory: eventlog_v2_appId(_[appAttemptId]) + * - The prefix of name on event files: events_[index]_[appId](_[appAttemptId])(.[codec]) + * - "index" would be monotonically increasing value (say, sequence) + * - The name of metadata (app. status) file name: appstatus_[appId](_[appAttemptId])(.inprogress) + * + * The writer will roll over the event log file when configured size is reached. Note that the + * writer doesn't check the size on file being open for write: the writer tracks the count of bytes + * written before compression is applied. + * + * For metadata files, the class will leverage zero-byte file, as it provides minimized cost. + */ +class RollingEventLogFilesWriter( + appId: String, + appAttemptId : Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration) + extends EventLogFileWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) { + + import RollingEventLogFilesWriter._ + + private val eventFileMaxLengthKiB = sparkConf.get(EVENT_LOG_ROLLED_EVENT_LOG_MAX_FILE_SIZE) + + private val logDirForAppPath = getAppEventLogDirPath(logBaseDir, appId, appAttemptId) + + // Only defined if the file system scheme is not local + private var hadoopDataStream: Option[FSDataOutputStream] = None + private var countingOutputStream: Option[CountingOutputStream] = None + private var writer: Option[PrintWriter] = None + + // seq and event log path will be updated soon in rollEventLogFile, which `start` will call + private var index: Long = 0L + private var currentEventLogFilePath: Path = logDirForAppPath + + override def start(): Unit = { + requireLogBaseDirAsDirectory() + + if (fileSystem.exists(logDirForAppPath) && shouldOverwrite) { + fileSystem.delete(logDirForAppPath, true) + } + + if (fileSystem.exists(logDirForAppPath)) { + throw new IOException(s"Target log directory already exists ($logDirForAppPath)") + } + + fileSystem.mkdirs(logDirForAppPath, EventLogFileWriter.LOG_FILE_PERMISSIONS) + createAppStatusFile(inProgress = true) + rollEventLogFile() + } + + override def writeEvent(eventJson: String, flushLogger: Boolean = false): Unit = { + writer.foreach { w => + val currentLen = countingOutputStream.get.getBytesWritten + if (currentLen + eventJson.length > eventFileMaxLengthKiB * 1024) { + rollEventLogFile() + } + } + + // scalastyle:off println + writer.foreach(_.println(eventJson)) + // scalastyle:on println + if (flushLogger) { + writer.foreach(_.flush()) + hadoopDataStream.foreach(_.hflush()) + } + } + + private def rollEventLogFile(): Unit = { + writer.foreach(_.close()) + + index += 1 + currentEventLogFilePath = getEventLogFilePath(logDirForAppPath, appId, appAttemptId, index, + compressionCodecName) + + val (hadoopStream, outputStream) = initLogFile(currentEventLogFilePath) + hadoopDataStream = hadoopStream + countingOutputStream = Some(new CountingOutputStream(outputStream)) + writer = Some(new PrintWriter( + new OutputStreamWriter(countingOutputStream.get, StandardCharsets.UTF_8))) + } + + override def stop(): Unit = { + writer.foreach(_.close()) + val appStatusPathIncomplete = getAppStatusFilePath(logDirForAppPath, appId, appAttemptId, + inProgress = true) + val appStatusPathComplete = getAppStatusFilePath(logDirForAppPath, appId, appAttemptId, + inProgress = false) + renameFile(appStatusPathIncomplete, appStatusPathComplete, overwrite = true) + } + + override def logPath: String = logDirForAppPath.toString + + private def createAppStatusFile(inProgress: Boolean): Unit = { + val appStatusPath = getAppStatusFilePath(logDirForAppPath, appId, appAttemptId, inProgress) + val outputStream = fileSystem.create(appStatusPath) + // we intentionally create zero-byte file to minimize the cost + outputStream.close() + } +} + +object RollingEventLogFilesWriter { + def getAppEventLogDirPath(logBaseDir: URI, appId: String, appAttemptId: Option[String]): Path = + new Path(new Path(logBaseDir), "eventlog_v2_" + + EventLogFileWriter.nameForAppAndAttempt(appId, appAttemptId)) + + def getAppStatusFilePath( + appLogDir: Path, + appId: String, + appAttemptId: Option[String], + inProgress: Boolean): Path = { + val base = "appstatus_" + EventLogFileWriter.nameForAppAndAttempt(appId, appAttemptId) + val name = if (inProgress) base + EventLogFileWriter.IN_PROGRESS else base + new Path(appLogDir, name) + } + + def getEventLogFilePath( + appLogDir: Path, + appId: String, + appAttemptId: Option[String], + index: Long, + codecName: Option[String]): Path = { + val base = s"events_${index}_" + EventLogFileWriter.nameForAppAndAttempt(appId, appAttemptId) + val codec = codecName.map("." + _).getOrElse("") + new Path(appLogDir, base + codec) + } + + def isEventLogDir(status: FileStatus): Boolean = { + status.isDirectory && status.getPath.getName.startsWith("eventlog_v2_") + } + + def isEventLogFile(status: FileStatus): Boolean = { + status.isFile && isEventLogFile(status.getPath) + } + + def isEventLogFile(path: Path): Boolean = { + path.getName.startsWith("events_") + } + + def isAppStatusFile(status: FileStatus): Boolean = { + status.isFile && isAppStatusFile(status.getPath) + } + + def isAppStatusFile(path: Path): Boolean = { + path.getName.startsWith("appstatus") + } + + def getSequence(eventLogFileName: String): Long = { + require(eventLogFileName.startsWith("events_"), "Not a event log file!") + val seq = eventLogFileName.stripPrefix("events_").split("_")(0) + seq.toLong + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index dce9581be290..115eb19751bd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -439,27 +439,27 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) logDebug(s"Scanning $logDir with lastScanTime==$lastScanTime") val updated = Option(fs.listStatus(new Path(logDir))).map(_.toSeq).getOrElse(Nil) - .filter { entry => - !entry.isDirectory() && - // FsHistoryProvider used to generate a hidden file which can't be read. Accidentally - // reading a garbage file is safe, but we would log an error which can be scary to - // the end-user. - !entry.getPath().getName().startsWith(".") && - !isBlacklisted(entry.getPath) - } - .filter { entry => + .filter { entry => !isBlacklisted(entry.getPath) } + .flatMap { entry => EventLogFileReader(fs, entry) } + .filter { reader => try { - val info = listing.read(classOf[LogInfo], entry.getPath().toString()) + val info = listing.read(classOf[LogInfo], reader.rootPath.toString()) if (info.appId.isDefined) { // If the SHS view has a valid application, update the time the file was last seen so // that the entry is not deleted from the SHS listing. Also update the file size, in // case the code below decides we don't need to parse the log. - listing.write(info.copy(lastProcessed = newLastScanTime, fileSize = entry.getLen())) + listing.write(info.copy(lastProcessed = newLastScanTime, + fileSize = reader.fileSizeForLastIndex, + lastSequenceNum = reader.lastIndex, + isComplete = reader.completed)) } - if (shouldReloadLog(info, entry)) { - if (info.appId.isDefined && fastInProgressParsing) { + if (shouldReloadLog(info, reader)) { + // ignore fastInProgressParsing when the status of application is changed from + // in-progress to completed, which is needed for rolling event log. + if (info.appId.isDefined && (info.isComplete == reader.completed) && + fastInProgressParsing) { // When fast in-progress parsing is on, we don't need to re-parse when the // size changes, but we do need to invalidate any existing UIs. // Also, we need to update the `lastUpdated time` to display the updated time in @@ -472,6 +472,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) attempt.info.copy(lastUpdated = new Date(newLastScanTime)), attempt.logPath, attempt.fileSize, + attempt.lastSequenceNum, attempt.adminAcls, attempt.viewAcls, attempt.adminAclsGroups, @@ -497,24 +498,25 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // If the file is currently not being tracked by the SHS, add an entry for it and try // to parse it. This will allow the cleaner code to detect the file as stale later on // if it was not possible to parse it. - listing.write(LogInfo(entry.getPath().toString(), newLastScanTime, LogType.EventLogs, - None, None, entry.getLen())) - entry.getLen() > 0 + listing.write(LogInfo(reader.rootPath.toString(), newLastScanTime, LogType.EventLogs, + None, None, reader.fileSizeForLastIndex, reader.lastIndex, + reader.completed)) + reader.fileSizeForLastIndex > 0 } } .sortWith { case (entry1, entry2) => - entry1.getModificationTime() > entry2.getModificationTime() + entry1.modificationTime > entry2.modificationTime } if (updated.nonEmpty) { - logDebug(s"New/updated attempts found: ${updated.size} ${updated.map(_.getPath)}") + logDebug(s"New/updated attempts found: ${updated.size} ${updated.map(_.rootPath)}") } val tasks = updated.flatMap { entry => try { val task: Future[Unit] = replayExecutor.submit( () => mergeApplicationListing(entry, newLastScanTime, true)) - Some(task -> entry.getPath) + Some(task -> entry.rootPath) } catch { // let the iteration over the updated entries break, since an exception on // replayExecutor.submit (..) indicates the ExecutorService is unable @@ -574,22 +576,27 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } - private[history] def shouldReloadLog(info: LogInfo, entry: FileStatus): Boolean = { - var result = info.fileSize < entry.getLen - if (!result && info.logPath.endsWith(EventLoggingListener.IN_PROGRESS)) { - try { - result = Utils.tryWithResource(fs.open(entry.getPath)) { in => - in.getWrappedStream match { - case dfsIn: DFSInputStream => info.fileSize < dfsIn.getFileLength - case _ => false - } + private[history] def shouldReloadLog(info: LogInfo, reader: EventLogFileReader): Boolean = { + if (info.isComplete != reader.completed) { + true + } else { + var result = if (info.lastSequenceNum.isDefined) { + require(reader.lastIndex.isDefined) + info.lastSequenceNum.get < reader.lastIndex.get || + info.fileSize < reader.fileSizeForLastIndex + } else { + info.fileSize < reader.fileSizeForLastIndex + } + if (!result && !reader.completed) { + try { + result = reader.fileSizeForLastIndexForDFS.exists(info.fileSize < _) + } catch { + case e: Exception => + logDebug(s"Failed to check the length for the file : ${info.logPath}", e) } - } catch { - case e: Exception => - logDebug(s"Failed to check the length for the file : ${info.logPath}", e) } + result } - result } private def cleanAppData(appId: String, attemptId: Option[String], logPath: String): Unit = { @@ -636,23 +643,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) attemptId: Option[String], zipStream: ZipOutputStream): Unit = { - /** - * This method compresses the files passed in, and writes the compressed data out into the - * [[OutputStream]] passed in. Each file is written as a new [[ZipEntry]] with its name being - * the name of the file being compressed. - */ - def zipFileToStream(file: Path, entryName: String, outputStream: ZipOutputStream): Unit = { - val fs = file.getFileSystem(hadoopConf) - val inputStream = fs.open(file, 1 * 1024 * 1024) // 1MB Buffer - try { - outputStream.putNextEntry(new ZipEntry(entryName)) - ByteStreams.copy(inputStream, outputStream) - outputStream.closeEntry() - } finally { - inputStream.close() - } - } - val app = try { load(appId) } catch { @@ -665,9 +655,10 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) attemptId .map { id => app.attempts.filter(_.info.attemptId == Some(id)) } .getOrElse(app.attempts) - .map(_.logPath) - .foreach { log => - zipFileToStream(new Path(logDir, log), log, zipStream) + .foreach { attempt => + val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), + attempt.lastSequenceNum) + reader.zipEventLogFiles(zipStream) } } finally { zipStream.close() @@ -678,7 +669,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) * Replay the given log file, saving the application in the listing db. */ protected def mergeApplicationListing( - fileStatus: FileStatus, + reader: EventLogFileReader, scanTime: Long, enableOptimizations: Boolean): Unit = { val eventsFilter: ReplayEventsFilter = { eventString => @@ -688,8 +679,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) eventString.startsWith(ENV_UPDATE_EVENT_PREFIX) } - val logPath = fileStatus.getPath() - val appCompleted = isCompleted(logPath.getName()) + val logPath = reader.rootPath + val appCompleted = reader.completed val reparseChunkSize = conf.get(END_EVENT_REPARSE_CHUNK_SIZE) // Enable halt support in listener if: @@ -699,12 +690,15 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) ((!appCompleted && fastInProgressParsing) || reparseChunkSize > 0) val bus = new ReplayListenerBus() - val listener = new AppListingListener(fileStatus, clock, shouldHalt) + val listener = new AppListingListener(reader, clock, shouldHalt) bus.addListener(listener) logInfo(s"Parsing $logPath for listing data...") - Utils.tryWithResource(EventLoggingListener.openEventLog(logPath, fs)) { in => - bus.replay(in, logPath.toString, !appCompleted, eventsFilter) + val logFiles = reader.listEventLogFiles + logFiles.foreach { file => + Utils.tryWithResource(EventLogFileReader.openEventLog(file.getPath, fs)) { in => + bus.replay(in, file.getPath.toString, !appCompleted, eventsFilter) + } } // If enabled above, the listing listener will halt parsing when there's enough information to @@ -727,8 +721,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // current position is, since the replay listener bus buffers data internally. val lookForEndEvent = shouldHalt && (appCompleted || !fastInProgressParsing) if (lookForEndEvent && listener.applicationInfo.isDefined) { - Utils.tryWithResource(EventLoggingListener.openEventLog(logPath, fs)) { in => - val target = fileStatus.getLen() - reparseChunkSize + val lastFile = logFiles.last + Utils.tryWithResource(EventLogFileReader.openEventLog(lastFile.getPath, fs)) { in => + val target = lastFile.getLen - reparseChunkSize if (target > 0) { logInfo(s"Looking for end event; skipping $target bytes from $logPath...") var skipped = 0L @@ -745,7 +740,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) source.next() } - bus.replay(source, logPath.toString, !appCompleted, eventsFilter) + bus.replay(source, lastFile.getPath.toString, !appCompleted, eventsFilter) } } @@ -758,12 +753,15 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) invalidateUI(app.info.id, app.attempts.head.info.attemptId) addListing(app) listing.write(LogInfo(logPath.toString(), scanTime, LogType.EventLogs, Some(app.info.id), - app.attempts.head.info.attemptId, fileStatus.getLen())) + app.attempts.head.info.attemptId, reader.fileSizeForLastIndex, + reader.lastIndex, reader.completed)) // For a finished log, remove the corresponding "in progress" entry from the listing DB if // the file is really gone. - if (appCompleted) { - val inProgressLog = logPath.toString() + EventLoggingListener.IN_PROGRESS + // The logic is only valid for single event log, as root path doesn't change for + // rolled event logs. + if (appCompleted && reader.lastIndex.isDefined) { + val inProgressLog = logPath.toString() + EventLogFileWriter.IN_PROGRESS try { // Fetch the entry first to avoid an RPC when it's already removed. listing.read(classOf[LogInfo], inProgressLog) @@ -780,14 +778,15 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // mean the end event is before the configured threshold, so call the method again to // re-parse the whole log. logInfo(s"Reparsing $logPath since end event was not found.") - mergeApplicationListing(fileStatus, scanTime, false) + mergeApplicationListing(reader, scanTime, enableOptimizations = false) case _ => // If the app hasn't written down its app ID to the logs, still record the entry in the // listing db, with an empty ID. This will make the log eligible for deletion if the app // does not make progress after the configured max log age. listing.write( - LogInfo(logPath.toString(), scanTime, LogType.EventLogs, None, None, fileStatus.getLen())) + LogInfo(logPath.toString(), scanTime, LogType.EventLogs, None, None, + reader.fileSizeForLastIndex, reader.lastIndex, reader.completed)) } } @@ -922,7 +921,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) case e: NoSuchElementException => // For every new driver log file discovered, create a new entry in listing listing.write(LogInfo(f.getPath().toString(), currentTime, LogType.DriverLogs, None, - None, f.getLen())) + None, f.getLen(), None, false)) false } if (deleteFile) { @@ -953,7 +952,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) */ private def rebuildAppStore( store: KVStore, - eventLog: FileStatus, + reader: EventLogFileReader, lastUpdated: Long): Unit = { // Disable async updates, since they cause higher memory usage, and it's ok to take longer // to parse the event logs in the SHS. @@ -970,13 +969,15 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } replayBus.addListener(listener) try { - val path = eventLog.getPath() - logInfo(s"Parsing $path to re-build UI...") - Utils.tryWithResource(EventLoggingListener.openEventLog(path, fs)) { in => - replayBus.replay(in, path.toString(), maybeTruncated = !isCompleted(path.toString())) + logInfo(s"Parsing ${reader.rootPath} to re-build UI...") + val logFiles = reader.listEventLogFiles + logFiles.foreach { file => + Utils.tryWithResource(EventLogFileReader.openEventLog(file.getPath, fs)) { in => + replayBus.replay(in, file.getPath.toString, maybeTruncated = !reader.completed) + } } trackingStore.close(false) - logInfo(s"Finished parsing $path") + logInfo(s"Finished parsing ${reader.rootPath}") } catch { case e: Exception => Utils.tryLogNonFatalError { @@ -1067,15 +1068,15 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // At this point the disk data either does not exist or was deleted because it failed to // load, so the event log needs to be replayed. - val status = fs.getFileStatus(new Path(logDir, attempt.logPath)) - val isCompressed = EventLoggingListener.codecName(status.getPath()).flatMap { name => - Try(CompressionCodec.getShortName(name)).toOption - }.isDefined + + val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), + attempt.lastSequenceNum) + val isCompressed = reader.compressionCodec.isDefined logInfo(s"Leasing disk manager space for app $appId / ${attempt.info.attemptId}...") - val lease = dm.lease(status.getLen(), isCompressed) + val lease = dm.lease(reader.totalSize, isCompressed) val newStorePath = try { Utils.tryWithResource(KVUtils.open(lease.tmpPath, metadata)) { store => - rebuildAppStore(store, status, attempt.info.lastUpdated.getTime()) + rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime()) } lease.commit(appId, attempt.info.attemptId) } catch { @@ -1089,8 +1090,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private def createInMemoryStore(attempt: AttemptInfoWrapper): KVStore = { val store = new InMemoryStore() - val status = fs.getFileStatus(new Path(logDir, attempt.logPath)) - rebuildAppStore(store, status, attempt.info.lastUpdated.getTime()) + val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), + attempt.lastSequenceNum) + rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime()) store } @@ -1120,11 +1122,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } deleted } - - private def isCompleted(name: String): Boolean = { - !name.endsWith(EventLoggingListener.IN_PROGRESS) - } - } private[history] object FsHistoryProvider { @@ -1165,12 +1162,15 @@ private[history] case class LogInfo( logType: LogType.Value, appId: Option[String], attemptId: Option[String], - fileSize: Long) + fileSize: Long, + lastSequenceNum: Option[Long], + isComplete: Boolean) private[history] class AttemptInfoWrapper( val info: ApplicationAttemptInfo, val logPath: String, val fileSize: Long, + val lastSequenceNum: Option[Long], val adminAcls: Option[String], val viewAcls: Option[String], val adminAclsGroups: Option[String], @@ -1194,12 +1194,13 @@ private[history] class ApplicationInfoWrapper( } private[history] class AppListingListener( - log: FileStatus, + reader: EventLogFileReader, clock: Clock, haltEnabled: Boolean) extends SparkListener { private val app = new MutableApplicationInfo() - private val attempt = new MutableAttemptInfo(log.getPath().getName(), log.getLen()) + private val attempt = new MutableAttemptInfo(reader.rootPath.getName(), + reader.fileSizeForLastIndex, reader.lastIndex) private var gotEnvUpdate = false private var halted = false @@ -1218,7 +1219,7 @@ private[history] class AppListingListener( override def onApplicationEnd(event: SparkListenerApplicationEnd): Unit = { attempt.endTime = new Date(event.time) - attempt.lastUpdated = new Date(log.getModificationTime()) + attempt.lastUpdated = new Date(reader.modificationTime) attempt.duration = event.time - attempt.startTime.getTime() attempt.completed = true } @@ -1284,7 +1285,7 @@ private[history] class AppListingListener( } - private class MutableAttemptInfo(logPath: String, fileSize: Long) { + private class MutableAttemptInfo(logPath: String, fileSize: Long, lastSequence: Option[Long]) { var attemptId: Option[String] = None var startTime = new Date(-1) var endTime = new Date(-1) @@ -1313,6 +1314,7 @@ private[history] class AppListingListener( apiInfo, logPath, fileSize, + lastSequence, adminAcls, viewAcls, adminAclsGroups, diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index d142d2292972..f6c21a6d6de9 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -179,6 +179,17 @@ package object config { private[spark] val EVENT_LOG_CALLSITE_LONG_FORM = ConfigBuilder("spark.eventLog.longForm.enabled").booleanConf.createWithDefault(false) + private[spark] val EVENT_LOG_ENABLE_ROLLING = + ConfigBuilder("spark.eventLog.logRolling.enabled") + .doc("Whether rolling over event log files is enabled.") + .booleanConf.createWithDefault(false) + + private[spark] val EVENT_LOG_ROLLED_EVENT_LOG_MAX_FILE_SIZE = + ConfigBuilder("spark.eventLog.logRolling.maxFileSize") + .doc("The max size of event log file to be rolled over.") + .bytesConf(ByteUnit.KiB) + .createWithDefaultString("128m") + private[spark] val EXECUTOR_ID = ConfigBuilder("spark.executor.id").stringConf.createOptional diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index a0a4428dc7f5..7937f271bd77 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -17,24 +17,20 @@ package org.apache.spark.scheduler -import java.io._ import java.net.URI -import java.nio.charset.StandardCharsets -import scala.collection.mutable.{ArrayBuffer, Map} +import scala.collection.mutable import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} -import org.apache.hadoop.fs.permission.FsPermission import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SPARK_VERSION, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.EventLogFileWriter import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{JsonProtocol, Utils} /** @@ -42,13 +38,12 @@ import org.apache.spark.util.{JsonProtocol, Utils} * * Event logging is specified by the following configurable parameters: * spark.eventLog.enabled - Whether event logging is enabled. - * spark.eventLog.logBlockUpdates.enabled - Whether to log block updates - * spark.eventLog.compress - Whether to compress logged events - * spark.eventLog.compression.codec - The codec to compress logged events - * spark.eventLog.overwrite - Whether to overwrite any existing files. * spark.eventLog.dir - Path to the directory in which events are logged. - * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams + * spark.eventLog.logBlockUpdates.enabled - Whether to log block updates * spark.eventLog.logStageExecutorMetrics.enabled - Whether to log stage executor metrics + * + * Event log file writer maintains its own parameters: refer the javadoc of [[EventLogFileWriter]] + * and its descendant for more details. */ private[spark] class EventLoggingListener( appId: String, @@ -64,93 +59,43 @@ private[spark] class EventLoggingListener( this(appId, appAttemptId, logBaseDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) - private val shouldCompress = sparkConf.get(EVENT_LOG_COMPRESS) - private val shouldOverwrite = sparkConf.get(EVENT_LOG_OVERWRITE) - private val shouldLogBlockUpdates = sparkConf.get(EVENT_LOG_BLOCK_UPDATES) - private val shouldLogStageExecutorMetrics = sparkConf.get(EVENT_LOG_STAGE_EXECUTOR_METRICS) - private val testing = sparkConf.get(EVENT_LOG_TESTING) - private val outputBufferSize = sparkConf.get(EVENT_LOG_OUTPUT_BUFFER_SIZE).toInt - private val fileSystem = Utils.getHadoopFileSystem(logBaseDir, hadoopConf) - private val compressionCodec = - if (shouldCompress) { - Some(CompressionCodec.createCodec(sparkConf, sparkConf.get(EVENT_LOG_COMPRESSION_CODEC))) - } else { - None - } - // Visible for tests only. - private[scheduler] val compressionCodecName = compressionCodec.map { c => - CompressionCodec.getShortName(c.getClass.getName) - } - - // Only defined if the file system scheme is not local - private var hadoopDataStream: Option[FSDataOutputStream] = None - - private var writer: Option[PrintWriter] = None + // For testing. + private[scheduler] val logWriter: EventLogFileWriter = + EventLogFileWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) // For testing. Keep track of all JSON serialized events that have been logged. - private[scheduler] val loggedEvents = new ArrayBuffer[JValue] + private[scheduler] val loggedEvents = new mutable.ArrayBuffer[JValue] - // Visible for tests only. - private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) + private val shouldLogBlockUpdates = sparkConf.get(EVENT_LOG_BLOCK_UPDATES) + private val shouldLogStageExecutorMetrics = sparkConf.get(EVENT_LOG_STAGE_EXECUTOR_METRICS) + private val testing = sparkConf.get(EVENT_LOG_TESTING) // map of (stageId, stageAttempt) to executor metric peaks per executor/driver for the stage - private val liveStageExecutorMetrics = Map.empty[(Int, Int), Map[String, ExecutorMetrics]] + private val liveStageExecutorMetrics = + mutable.HashMap.empty[(Int, Int), mutable.HashMap[String, ExecutorMetrics]] /** * Creates the log file in the configured log directory. */ def start(): Unit = { - if (!fileSystem.getFileStatus(new Path(logBaseDir)).isDirectory) { - throw new IllegalArgumentException(s"Log directory $logBaseDir is not a directory.") - } - - val workingPath = logPath + IN_PROGRESS - val path = new Path(workingPath) - val uri = path.toUri - val defaultFs = FileSystem.getDefaultUri(hadoopConf).getScheme - val isDefaultLocal = defaultFs == null || defaultFs == "file" - - if (shouldOverwrite && fileSystem.delete(path, true)) { - logWarning(s"Event log $path already exists. Overwriting...") - } - - /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). - * Therefore, for local files, use FileOutputStream instead. */ - val dstream = - if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { - new FileOutputStream(uri.getPath) - } else { - hadoopDataStream = Some( - SparkHadoopUtil.createFile(fileSystem, path, sparkConf.get(EVENT_LOG_ALLOW_EC))) - hadoopDataStream.get - } + logWriter.start() + initEventLog() + } - try { - val cstream = compressionCodec.map(_.compressedContinuousOutputStream(dstream)) - .getOrElse(dstream) - val bstream = new BufferedOutputStream(cstream, outputBufferSize) - - EventLoggingListener.initEventLog(bstream, testing, loggedEvents) - fileSystem.setPermission(path, LOG_FILE_PERMISSIONS) - writer = Some(new PrintWriter(new OutputStreamWriter(bstream, StandardCharsets.UTF_8))) - logInfo("Logging events to %s".format(logPath)) - } catch { - case e: Exception => - dstream.close() - throw e + private def initEventLog(): Unit = { + val metadata = SparkListenerLogStart(SPARK_VERSION) + val eventJson = JsonProtocol.logStartToJson(metadata) + val metadataJson = compact(eventJson) + logWriter.writeEvent(metadataJson, flushLogger = true) + if (testing && loggedEvents != null) { + loggedEvents += eventJson } } /** Log the event as JSON. */ private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false): Unit = { val eventJson = JsonProtocol.sparkEventToJson(event) - // scalastyle:off println - writer.foreach(_.println(compact(render(eventJson)))) - // scalastyle:on println - if (flushLogger) { - writer.foreach(_.flush()) - hadoopDataStream.foreach(_.hflush()) - } + logWriter.writeEvent(compact(render(eventJson)), flushLogger) if (testing) { loggedEvents += eventJson } @@ -162,7 +107,7 @@ private[spark] class EventLoggingListener( if (shouldLogStageExecutorMetrics) { // record the peak metrics for the new stage liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()), - Map.empty[String, ExecutorMetrics]) + mutable.HashMap.empty[String, ExecutorMetrics]) } } @@ -296,32 +241,9 @@ private[spark] class EventLoggingListener( } } - /** - * Stop logging events. The event log file will be renamed so that it loses the - * ".inprogress" suffix. - */ + /** Stop logging events. */ def stop(): Unit = { - writer.foreach(_.close()) - - val target = new Path(logPath) - if (fileSystem.exists(target)) { - if (shouldOverwrite) { - logWarning(s"Event log $target already exists. Overwriting...") - if (!fileSystem.delete(target, true)) { - logWarning(s"Error deleting $target") - } - } else { - throw new IOException("Target log file already exists (%s)".format(logPath)) - } - } - fileSystem.rename(new Path(logPath + IN_PROGRESS), target) - // touch file to ensure modtime is current across those filesystems where rename() - // does not set it, -and which support setTimes(); it's a no-op on most object stores - try { - fileSystem.setTimes(target, System.currentTimeMillis(), -1) - } catch { - case e: Exception => logDebug(s"failed to set time of $target", e) - } + logWriter.stop() } private[spark] def redactEvent( @@ -342,93 +264,7 @@ private[spark] class EventLoggingListener( } private[spark] object EventLoggingListener extends Logging { - // Suffix applied to the names of files still being written by applications. - val IN_PROGRESS = ".inprogress" val DEFAULT_LOG_DIR = "/tmp/spark-events" // Dummy stage key used by driver in executor metrics updates val DRIVER_STAGE_KEY = (-1, -1) - - private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) - - // A cache for compression codecs to avoid creating the same codec many times - private val codecMap = Map.empty[String, CompressionCodec] - - /** - * Write metadata about an event log to the given stream. - * The metadata is encoded in the first line of the event log as JSON. - * - * @param logStream Raw output stream to the event log file. - */ - def initEventLog( - logStream: OutputStream, - testing: Boolean, - loggedEvents: ArrayBuffer[JValue]): Unit = { - val metadata = SparkListenerLogStart(SPARK_VERSION) - val eventJson = JsonProtocol.logStartToJson(metadata) - val metadataJson = compact(eventJson) + "\n" - logStream.write(metadataJson.getBytes(StandardCharsets.UTF_8)) - if (testing && loggedEvents != null) { - loggedEvents += eventJson - } - } - - /** - * Return a file-system-safe path to the log file for the given application. - * - * Note that because we currently only create a single log file for each application, - * we must encode all the information needed to parse this event log in the file name - * instead of within the file itself. Otherwise, if the file is compressed, for instance, - * we won't know which codec to use to decompress the metadata needed to open the file in - * the first place. - * - * The log file name will identify the compression codec used for the contents, if any. - * For example, app_123 for an uncompressed log, app_123.lzf for an LZF-compressed log. - * - * @param logBaseDir Directory where the log file will be written. - * @param appId A unique app ID. - * @param appAttemptId A unique attempt id of appId. May be the empty string. - * @param compressionCodecName Name to identify the codec used to compress the contents - * of the log, or None if compression is not enabled. - * @return A path which consists of file-system-safe characters. - */ - def getLogPath( - logBaseDir: URI, - appId: String, - appAttemptId: Option[String], - compressionCodecName: Option[String] = None): String = { - val base = new Path(logBaseDir).toString.stripSuffix("/") + "/" + Utils.sanitizeDirName(appId) - val codec = compressionCodecName.map("." + _).getOrElse("") - if (appAttemptId.isDefined) { - base + "_" + Utils.sanitizeDirName(appAttemptId.get) + codec - } else { - base + codec - } - } - - /** - * Opens an event log file and returns an input stream that contains the event data. - * - * @return input stream that holds one JSON record per line. - */ - def openEventLog(log: Path, fs: FileSystem): InputStream = { - val in = new BufferedInputStream(fs.open(log)) - try { - val codec = codecName(log).map { c => - codecMap.getOrElseUpdate(c, CompressionCodec.createCodec(new SparkConf, c)) - } - codec.map(_.compressedContinuousInputStream(in)).getOrElse(in) - } catch { - case e: Throwable => - in.close() - throw e - } - } - - def codecName(log: Path): Option[String] = { - // Compression codec is encoded as an extension, e.g. app_123.lzf - // Since we sanitize the app ID to not include periods, it is safe to split on it - val logName = log.getName.stripSuffix(IN_PROGRESS) - logName.split("\\.").tail.lastOption - } - } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 5b81671edb14..9244ab407cb6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -39,11 +39,11 @@ import org.apache.spark.TestUtils.JavaSourceFromString import org.apache.spark.api.r.RUtils import org.apache.spark.deploy.SparkSubmit._ import org.apache.spark.deploy.SparkSubmitUtils.MavenCoordinate +import org.apache.spark.deploy.history.EventLogFileReader import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher -import org.apache.spark.scheduler.EventLoggingListener import org.apache.spark.util.{CommandLineUtils, ResetSystemProperties, Utils} trait TestPrematureExit { @@ -535,7 +535,7 @@ class SparkSubmitSuite unusedJar.toString) runSparkSubmit(args) val listStatus = fileSystem.listStatus(testDirPath) - val logData = EventLoggingListener.openEventLog(listStatus.last.getPath, fileSystem) + val logData = EventLogFileReader.openEventLog(listStatus.last.getPath, fileSystem) Source.fromInputStream(logData).getLines().foreach { line => assert(!line.contains("secret_password")) } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala new file mode 100644 index 000000000000..5bec9b49209f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala @@ -0,0 +1,345 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File} +import java.net.URI +import java.nio.charset.StandardCharsets +import java.util.zip.{ZipInputStream, ZipOutputStream} + +import scala.collection.mutable + +import com.google.common.io.{ByteStreams, Files} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.EventLogTestHelper._ +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.Utils + + +abstract class EventLogFileReadersSuite extends SparkFunSuite with LocalSparkContext + with BeforeAndAfter with Logging { + + protected val fileSystem = Utils.getHadoopFileSystem("/", + SparkHadoopUtil.get.newConfiguration(new SparkConf())) + protected var testDir: File = _ + protected var testDirPath: Path = _ + + before { + testDir = Utils.createTempDir(namePrefix = s"event log") + testDir.deleteOnExit() + testDirPath = new Path(testDir.getAbsolutePath()) + } + + after { + Utils.deleteRecursively(testDir) + } + + test("Retrieve EventLogFileReader correctly") { + def assertInstanceOfEventLogReader( + expectedClazz: Option[Class[_ <: EventLogFileReader]], + actual: Option[EventLogFileReader]): Unit = { + if (expectedClazz.isEmpty) { + assert(actual.isEmpty, s"Expected no EventLogFileReader instance but was " + + s"${actual.map(_.getClass).getOrElse("")}") + } else { + assert(actual.isDefined, s"Expected an EventLogFileReader instance but was empty") + assert(expectedClazz.get.isAssignableFrom(actual.get.getClass), + s"Expected ${expectedClazz.get} but was ${actual.get.getClass}") + } + } + + def testForPathWithoutSeq( + path: Path, + isFile: Boolean, + expectedClazz: Option[Class[_ <: EventLogFileReader]]): Unit = { + if (isFile) { + Utils.tryWithResource(fileSystem.create(path)) { is => + is.writeInt(10) + } + } else { + fileSystem.mkdirs(path) + } + + val reader = EventLogFileReader(fileSystem, path) + assertInstanceOfEventLogReader(expectedClazz, reader) + val reader2 = EventLogFileReader(fileSystem, + fileSystem.getFileStatus(path)) + assertInstanceOfEventLogReader(expectedClazz, reader) + } + + // path with no last index - single event log + val reader1 = EventLogFileReader(fileSystem, new Path(testDirPath, "aaa"), + None) + assertInstanceOfEventLogReader(Some(classOf[SingleFileEventLogFileReader]), Some(reader1)) + + // path with last index - rolling event log + val reader2 = EventLogFileReader(fileSystem, + new Path(testDirPath, "eventlog_v2_aaa"), Some(3)) + assertInstanceOfEventLogReader(Some(classOf[RollingEventLogFilesFileReader]), Some(reader2)) + + // path - file (both path and FileStatus) + val eventLogFile = new Path(testDirPath, "bbb") + testForPathWithoutSeq(eventLogFile, isFile = true, Some(classOf[SingleFileEventLogFileReader])) + + // path - file starting with "." + val invalidEventLogFile = new Path(testDirPath, ".bbb") + testForPathWithoutSeq(invalidEventLogFile, isFile = true, None) + + // path - directory with "eventlog_v2_" prefix + val eventLogDir = new Path(testDirPath, "eventlog_v2_ccc") + testForPathWithoutSeq(eventLogDir, isFile = false, + Some(classOf[RollingEventLogFilesFileReader])) + + // path - directory with no "eventlog_v2_" prefix + val invalidEventLogDir = new Path(testDirPath, "ccc") + testForPathWithoutSeq(invalidEventLogDir, isFile = false, None) + } + + val allCodecs = Seq(None) ++ + CompressionCodec.ALL_COMPRESSION_CODECS.map { c => Some(CompressionCodec.getShortName(c)) } + + allCodecs.foreach { codecShortName => + test(s"get information, list event log files, zip log files - with codec $codecShortName") { + val appId = getUniqueApplicationId + val attemptId = None + + val conf = getLoggingConf(testDirPath, codecShortName) + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + + val writer = createWriter(appId, attemptId, testDirPath.toUri, conf, hadoopConf) + writer.start() + + // The test for writing events into EventLogFileWriter is covered to its own test suite. + val dummyData = Seq("dummy1", "dummy2", "dummy3") + dummyData.foreach(writer.writeEvent(_, flushLogger = true)) + + val logPathIncompleted = getCurrentLogPath(writer.logPath, isCompleted = false) + val readerOpt = EventLogFileReader(fileSystem, + new Path(logPathIncompleted)) + assertAppropriateReader(readerOpt) + val reader = readerOpt.get + + verifyReader(reader, new Path(logPathIncompleted), codecShortName, isCompleted = false) + + writer.stop() + + val logPathCompleted = getCurrentLogPath(writer.logPath, isCompleted = true) + val readerOpt2 = EventLogFileReader(fileSystem, new Path(logPathCompleted)) + assertAppropriateReader(readerOpt2) + val reader2 = readerOpt2.get + + verifyReader(reader2, new Path(logPathCompleted), codecShortName, isCompleted = true) + } + } + + protected def createWriter( + appId: String, + appAttemptId : Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration): EventLogFileWriter + + protected def getCurrentLogPath(logPath: String, isCompleted: Boolean): String + + protected def assertAppropriateReader(actualReader: Option[EventLogFileReader]): Unit + + protected def verifyReader( + reader: EventLogFileReader, + logPath: Path, + compressionCodecShortName: Option[String], + isCompleted: Boolean): Unit +} + +class SingleFileEventLogFileReaderSuite extends EventLogFileReadersSuite { + override protected def createWriter( + appId: String, + appAttemptId: Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration): EventLogFileWriter = { + new SingleEventLogFileWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) + } + + override protected def assertAppropriateReader(actualReader: Option[EventLogFileReader]): Unit = { + assert(actualReader.isDefined, s"Expected an EventLogReader instance but was empty") + assert(actualReader.get.isInstanceOf[SingleFileEventLogFileReader], + s"Expected SingleFileEventLogReader but was ${actualReader.get.getClass}") + } + + override protected def getCurrentLogPath(logPath: String, isCompleted: Boolean): String = { + if (!isCompleted) logPath + EventLogFileWriter.IN_PROGRESS else logPath + } + + override protected def verifyReader( + reader: EventLogFileReader, + logPath: Path, + compressionCodecShortName: Option[String], + isCompleted: Boolean): Unit = { + val stats = fileSystem.getFileStatus(logPath) + + assert(stats.isFile) + assert(reader.rootPath === fileSystem.makeQualified(logPath)) + assert(reader.lastIndex.isEmpty) + assert(reader.fileSizeForLastIndex === stats.getLen) + assert(reader.completed === isCompleted) + assert(reader.modificationTime === stats.getModificationTime) + assert(reader.listEventLogFiles.length === 1) + assert(reader.listEventLogFiles.map(_.getPath.toUri.getPath) === + Seq(logPath.toUri.getPath)) + assert(reader.compressionCodec === compressionCodecShortName) + assert(reader.totalSize === stats.getLen) + + val underlyingStream = new ByteArrayOutputStream() + Utils.tryWithResource(new ZipOutputStream(underlyingStream)) { os => + reader.zipEventLogFiles(os) + } + + Utils.tryWithResource(new ZipInputStream( + new ByteArrayInputStream(underlyingStream.toByteArray))) { is => + + var entry = is.getNextEntry + assert(entry != null) + val actual = new String(ByteStreams.toByteArray(is), StandardCharsets.UTF_8) + val expected = Files.toString(new File(logPath.toString), StandardCharsets.UTF_8) + assert(actual === expected) + assert(is.getNextEntry === null) + } + } +} + +class RollingEventLogFilesReaderSuite extends EventLogFileReadersSuite { + allCodecs.foreach { codecShortName => + test(s"rolling event log files - codec $codecShortName") { + val appId = getUniqueApplicationId + val attemptId = None + + val conf = getLoggingConf(testDirPath, codecShortName) + conf.set(EVENT_LOG_ENABLE_ROLLING, true) + conf.set(EVENT_LOG_ROLLED_EVENT_LOG_MAX_FILE_SIZE.key, "1k") + + val writer = createWriter(appId, attemptId, testDirPath.toUri, conf, + SparkHadoopUtil.get.newConfiguration(conf)) + + writer.start() + + val dummyString = "dummy" + val dummyStringBytesLen = dummyString.getBytes(StandardCharsets.UTF_8).length + val expectedLines = mutable.ArrayBuffer[String]() + + // write log more than 2k (intended to roll over to 3 files) + val repeatCount = Math.floor((1024 * 2) / dummyStringBytesLen).toInt + (0 until repeatCount).foreach { _ => + expectedLines.append(dummyString) + writer.writeEvent(dummyString, flushLogger = true) + } + + val logPathIncompleted = getCurrentLogPath(writer.logPath, isCompleted = false) + val readerOpt = EventLogFileReader(fileSystem, + new Path(logPathIncompleted)) + verifyReader(readerOpt.get, new Path(logPathIncompleted), codecShortName, isCompleted = false) + assert(readerOpt.get.listEventLogFiles.length === 3) + + writer.stop() + + val logPathCompleted = getCurrentLogPath(writer.logPath, isCompleted = true) + val readerOpt2 = EventLogFileReader(fileSystem, new Path(logPathCompleted)) + verifyReader(readerOpt2.get, new Path(logPathCompleted), codecShortName, isCompleted = true) + assert(readerOpt.get.listEventLogFiles.length === 3) + } + } + + override protected def createWriter( + appId: String, + appAttemptId: Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration): EventLogFileWriter = { + new RollingEventLogFilesWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) + } + + override protected def assertAppropriateReader(actualReader: Option[EventLogFileReader]): Unit = { + assert(actualReader.isDefined, s"Expected an EventLogReader instance but was empty") + assert(actualReader.get.isInstanceOf[RollingEventLogFilesFileReader], + s"Expected RollingEventLogFilesReader but was ${actualReader.get.getClass}") + } + + override protected def getCurrentLogPath(logPath: String, isCompleted: Boolean): String = logPath + + override protected def verifyReader( + reader: EventLogFileReader, + logPath: Path, + compressionCodecShortName: Option[String], + isCompleted: Boolean): Unit = { + import RollingEventLogFilesWriter._ + + val stats = fileSystem.getFileStatus(logPath) + assert(stats.isDirectory) + + val statsInDir = fileSystem.listStatus(logPath) + val eventFiles = statsInDir.filter(isEventLogFile).sortBy(s => getSequence(s.getPath.getName)) + assert(eventFiles.nonEmpty) + val lastEventFile = eventFiles.last + val allLen = eventFiles.map(_.getLen).sum + + assert(reader.rootPath === fileSystem.makeQualified(logPath)) + assert(reader.lastIndex === Some(getSequence(lastEventFile.getPath.getName))) + assert(reader.fileSizeForLastIndex === lastEventFile.getLen) + assert(reader.completed === isCompleted) + assert(reader.modificationTime === lastEventFile.getModificationTime) + assert(reader.listEventLogFiles.length === eventFiles.length) + assert(reader.listEventLogFiles.map(_.getPath) === eventFiles.map(_.getPath)) + assert(reader.compressionCodec === compressionCodecShortName) + assert(reader.totalSize === allLen) + + val underlyingStream = new ByteArrayOutputStream() + Utils.tryWithResource(new ZipOutputStream(underlyingStream)) { os => + reader.zipEventLogFiles(os) + } + + Utils.tryWithResource(new ZipInputStream( + new ByteArrayInputStream(underlyingStream.toByteArray))) { is => + + var entry = is.getNextEntry + assert(entry != null) + + // directory + assert(entry.getName === logPath.getName + "/") + + fileSystem.listStatus(logPath).foreach { file => + entry = is.getNextEntry + assert(entry != null) + + assert(entry.getName === logPath.getName + "/" + file.getPath.getName) + + val actual = new String(ByteStreams.toByteArray(is), StandardCharsets.UTF_8) + val expected = Files.toString(new File(logPath.toString, file.getPath.getName), + StandardCharsets.UTF_8) + assert(actual === expected) + } + + assert(is.getNextEntry === null) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala new file mode 100644 index 000000000000..17755e1d546b --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala @@ -0,0 +1,400 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import java.io.{File, FileOutputStream, IOException} +import java.net.URI +import java.nio.charset.StandardCharsets + +import scala.collection.mutable +import scala.io.Source + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.EventLogTestHelper._ +import org.apache.spark.internal.config._ +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.Utils + + +abstract class EventLogFileWritersSuite extends SparkFunSuite with LocalSparkContext + with BeforeAndAfter { + + protected val fileSystem = Utils.getHadoopFileSystem("/", + SparkHadoopUtil.get.newConfiguration(new SparkConf())) + protected var testDir: File = _ + protected var testDirPath: Path = _ + + before { + testDir = Utils.createTempDir(namePrefix = s"event log") + testDir.deleteOnExit() + testDirPath = new Path(testDir.getAbsolutePath()) + } + + after { + Utils.deleteRecursively(testDir) + } + + test("create EventLogFileWriter with enable/disable rolling") { + def buildWriterAndVerify(conf: SparkConf, expectedClazz: Class[_]): Unit = { + val writer = EventLogFileWriter( + getUniqueApplicationId, None, testDirPath.toUri, conf, + SparkHadoopUtil.get.newConfiguration(conf)) + val writerClazz = writer.getClass + assert(expectedClazz === writerClazz, + s"default file writer should be $expectedClazz, but $writerClazz") + } + + val conf = new SparkConf + conf.set(EVENT_LOG_ENABLED, true) + conf.set(EVENT_LOG_DIR, testDir.toString) + + // default config + buildWriterAndVerify(conf, classOf[SingleEventLogFileWriter]) + + conf.set(EVENT_LOG_ENABLE_ROLLING, true) + buildWriterAndVerify(conf, classOf[RollingEventLogFilesWriter]) + + conf.set(EVENT_LOG_ENABLE_ROLLING, false) + buildWriterAndVerify(conf, classOf[SingleEventLogFileWriter]) + } + + val allCodecs = Seq(None) ++ + CompressionCodec.ALL_COMPRESSION_CODECS.map(c => Some(CompressionCodec.getShortName(c))) + + allCodecs.foreach { codecShortName => + test(s"initialize, write, stop - with codec $codecShortName") { + val appId = getUniqueApplicationId + val attemptId = None + + val conf = getLoggingConf(testDirPath, codecShortName) + val writer = createWriter(appId, attemptId, testDirPath.toUri, conf, + SparkHadoopUtil.get.newConfiguration(conf)) + + writer.start() + + // snappy stream throws exception on empty stream, so we should provide some data to test. + val dummyData = Seq("dummy1", "dummy2", "dummy3") + dummyData.foreach(writer.writeEvent(_, flushLogger = true)) + + verifyWriteEventLogFile(appId, attemptId, testDirPath.toUri, codecShortName, + isCompleted = false, dummyData) + + writer.stop() + + verifyWriteEventLogFile(appId, attemptId, testDirPath.toUri, codecShortName, + isCompleted = true, dummyData) + } + } + + test("spark.eventLog.compression.codec overrides spark.io.compression.codec") { + val conf = new SparkConf + conf.set(EVENT_LOG_COMPRESS, true) + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + + val appId = "test" + val appAttemptId = None + + // The default value is `spark.io.compression.codec`. + val writer = createWriter(appId, appAttemptId, testDirPath.toUri, conf, hadoopConf) + assert(writer.compressionCodecName.contains("lz4")) + + // `spark.eventLog.compression.codec` overrides `spark.io.compression.codec`. + conf.set(EVENT_LOG_COMPRESSION_CODEC, "zstd") + val writer2 = createWriter(appId, appAttemptId, testDirPath.toUri, conf, hadoopConf) + assert(writer2.compressionCodecName.contains("zstd")) + } + + protected def readLinesFromEventLogFile(log: Path, fs: FileSystem): List[String] = { + val logDataStream = EventLogFileReader.openEventLog(log, fs) + try { + Source.fromInputStream(logDataStream).getLines().toList + } finally { + logDataStream.close() + } + } + + protected def createWriter( + appId: String, + appAttemptId : Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration): EventLogFileWriter + + protected def verifyWriteEventLogFile( + appId: String, + appAttemptId : Option[String], + logBaseDir: URI, + compressionCodecShortName: Option[String], + isCompleted: Boolean, + expectedLines: Seq[String] = Seq.empty): Unit + + protected def skipVerifyEventLogFile( + compressionCodecShortName: Option[String], + isCompleted: Boolean): Boolean = { + // Spark initializes LZ4BlockOutputStream with syncFlush=false, so we can't force + // pending bytes to be flushed. It's only guaranteed when stream is closed, so + // we only check for lz4 when isCompleted = true. + + // zstd seems to have issue in reading while write stream is in progress of writing + !isCompleted && + (compressionCodecShortName.contains("lz4") || compressionCodecShortName.contains("zstd")) + } +} + +class SingleEventLogFileWriterSuite extends EventLogFileWritersSuite { + + test("Log overwriting") { + val appId = "test" + val appAttemptId = None + val logUri = SingleEventLogFileWriter.getLogPath(testDir.toURI, appId, appAttemptId) + + val conf = getLoggingConf(testDirPath) + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + val writer = createWriter(appId, appAttemptId, testDir.toURI, conf, hadoopConf) + + val logPath = new Path(logUri).toUri.getPath + writer.start() + + val dummyData = Seq("dummy1", "dummy2", "dummy3") + dummyData.foreach(writer.writeEvent(_, flushLogger = true)) + + // Create file before writing the event log + new FileOutputStream(new File(logPath)).close() + // Expected IOException, since we haven't enabled log overwrite. + intercept[IOException] { writer.stop() } + + // Try again, but enable overwriting. + conf.set(EVENT_LOG_OVERWRITE, true) + val writer2 = createWriter(appId, appAttemptId, testDir.toURI, conf, hadoopConf) + writer2.start() + dummyData.foreach(writer2.writeEvent(_, flushLogger = true)) + writer2.stop() + } + + test("Event log name") { + val baseDirUri = Utils.resolveURI("/base-dir") + // without compression + assert(s"${baseDirUri.toString}/app1" === SingleEventLogFileWriter.getLogPath( + baseDirUri, "app1", None, None)) + // with compression + assert(s"${baseDirUri.toString}/app1.lzf" === + SingleEventLogFileWriter.getLogPath(baseDirUri, "app1", None, Some("lzf"))) + // illegal characters in app ID + assert(s"${baseDirUri.toString}/a-fine-mind_dollar_bills__1" === + SingleEventLogFileWriter.getLogPath(baseDirUri, + "a fine:mind$dollar{bills}.1", None, None)) + // illegal characters in app ID with compression + assert(s"${baseDirUri.toString}/a-fine-mind_dollar_bills__1.lz4" === + SingleEventLogFileWriter.getLogPath(baseDirUri, + "a fine:mind$dollar{bills}.1", None, Some("lz4"))) + } + + override protected def createWriter( + appId: String, + appAttemptId: Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration): EventLogFileWriter = { + new SingleEventLogFileWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) + } + + override protected def verifyWriteEventLogFile( + appId: String, + appAttemptId: Option[String], + logBaseDir: URI, + compressionCodecShortName: Option[String], + isCompleted: Boolean, + expectedLines: Seq[String]): Unit = { + // read single event log file + val logPath = SingleEventLogFileWriter.getLogPath(logBaseDir, appId, appAttemptId, + compressionCodecShortName) + + val finalLogPath = if (!isCompleted) { + new Path(logPath + EventLogFileWriter.IN_PROGRESS) + } else { + new Path(logPath) + } + + assert(fileSystem.exists(finalLogPath) && fileSystem.isFile(finalLogPath)) + if (!skipVerifyEventLogFile(compressionCodecShortName, isCompleted)) { + assert(expectedLines === readLinesFromEventLogFile(finalLogPath, fileSystem)) + } + } +} + +class RollingEventLogFilesWriterSuite extends EventLogFileWritersSuite { + import RollingEventLogFilesWriter._ + + test("Event log names") { + val baseDirUri = Utils.resolveURI("/base-dir") + val appId = "app1" + val appAttemptId = None + + // happy case with app ID + val logDir = RollingEventLogFilesWriter.getAppEventLogDirPath(baseDirUri, appId, None) + assert(s"${baseDirUri.toString}/eventlog_v2_app1" === logDir.toString) + + // appstatus: inprogress or completed + assert(s"$logDir/appstatus_app1.inprogress" === + RollingEventLogFilesWriter.getAppStatusFilePath(logDir, appId, appAttemptId, + inProgress = true).toString) + assert(s"$logDir/appstatus_app1" === + RollingEventLogFilesWriter.getAppStatusFilePath(logDir, appId, appAttemptId, + inProgress = false).toString) + + // without compression + assert(s"$logDir/events_1_app1" === + RollingEventLogFilesWriter.getEventLogFilePath(logDir, appId, appAttemptId, 1, None).toString) + + // with compression + assert(s"$logDir/events_1_app1.lzf" === + RollingEventLogFilesWriter.getEventLogFilePath(logDir, appId, appAttemptId, + 1, Some("lzf")).toString) + + // illegal characters in app ID + assert(s"${baseDirUri.toString}/eventlog_v2_a-fine-mind_dollar_bills__1" === + RollingEventLogFilesWriter.getAppEventLogDirPath(baseDirUri, + "a fine:mind$dollar{bills}.1", None).toString) + } + + test("Log overwriting") { + val appId = "test" + val appAttemptId = None + val logDirPath = RollingEventLogFilesWriter.getAppEventLogDirPath(testDir.toURI, appId, + appAttemptId) + + val conf = getLoggingConf(testDirPath) + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + val writer = createWriter(appId, appAttemptId, testDir.toURI, conf, hadoopConf) + + val logPath = logDirPath.toUri.getPath + + // Create file before writing the event log directory + // it doesn't matter whether the existing one is file or directory + new FileOutputStream(new File(logPath)).close() + + // Expected IOException, since we haven't enabled log overwrite. + // Note that the place IOException is thrown is different from single event log file. + intercept[IOException] { writer.start() } + + // Try again, but enable overwriting. + conf.set(EVENT_LOG_OVERWRITE, true) + + val writer2 = createWriter(appId, appAttemptId, testDir.toURI, conf, hadoopConf) + writer2.start() + val dummyData = Seq("dummy1", "dummy2", "dummy3") + dummyData.foreach(writer2.writeEvent(_, flushLogger = true)) + writer2.stop() + } + + allCodecs.foreach { codecShortName => + test(s"rolling event log files - codec $codecShortName") { + def assertEventLogFilesSequence( + eventLogFiles: Seq[FileStatus], + expectedLastSequence: Int, + expectedMaxSizeBytes: Long): Unit = { + assert(eventLogFiles.forall(f => f.getLen < expectedMaxSizeBytes)) + assert((1 to expectedLastSequence) === + eventLogFiles.map(f => getSequence(f.getPath.getName))) + } + + val appId = getUniqueApplicationId + val attemptId = None + + val conf = getLoggingConf(testDirPath, codecShortName) + conf.set(EVENT_LOG_ENABLE_ROLLING, true) + conf.set(EVENT_LOG_ROLLED_EVENT_LOG_MAX_FILE_SIZE.key, "1k") + + val writer = createWriter(appId, attemptId, testDirPath.toUri, conf, + SparkHadoopUtil.get.newConfiguration(conf)) + + writer.start() + + val dummyString = "dummy" + val dummyStringBytesLen = dummyString.getBytes(StandardCharsets.UTF_8).length + val expectedLines = mutable.ArrayBuffer[String]() + + // write log more than 2k (intended to roll over to 3 files) + val repeatCount = Math.floor((1024 * 2) / dummyStringBytesLen).toInt + (0 until repeatCount).foreach { _ => + expectedLines.append(dummyString) + writer.writeEvent(dummyString, flushLogger = true) + } + + val logDirPath = getAppEventLogDirPath(testDirPath.toUri, appId, attemptId) + + val eventLogFiles = listEventLogFiles(logDirPath) + assertEventLogFilesSequence(eventLogFiles, 3, 1024 * 1024) + + verifyWriteEventLogFile(appId, attemptId, testDirPath.toUri, + codecShortName, isCompleted = false, expectedLines) + + writer.stop() + + val eventLogFiles2 = listEventLogFiles(logDirPath) + assertEventLogFilesSequence(eventLogFiles2, 3, 1024 * 1024) + + verifyWriteEventLogFile(appId, attemptId, testDirPath.toUri, + codecShortName, isCompleted = true, expectedLines) + } + } + + override protected def createWriter( + appId: String, + appAttemptId: Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration): EventLogFileWriter = { + new RollingEventLogFilesWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) + } + + override protected def verifyWriteEventLogFile( + appId: String, + appAttemptId: Option[String], + logBaseDir: URI, + compressionCodecShortName: Option[String], + isCompleted: Boolean, + expectedLines: Seq[String]): Unit = { + val logDirPath = getAppEventLogDirPath(logBaseDir, appId, appAttemptId) + + assert(fileSystem.exists(logDirPath) && fileSystem.isDirectory(logDirPath)) + + val appStatusFile = getAppStatusFilePath(logDirPath, appId, appAttemptId, !isCompleted) + assert(fileSystem.exists(appStatusFile) && fileSystem.isFile(appStatusFile)) + + val eventLogFiles = listEventLogFiles(logDirPath) + if (!skipVerifyEventLogFile(compressionCodecShortName, isCompleted)) { + val allLines = mutable.ArrayBuffer[String]() + eventLogFiles.foreach { file => + allLines.appendAll(readLinesFromEventLogFile(file.getPath, fileSystem)) + } + + assert(expectedLines === allLines) + } + } + + private def listEventLogFiles(logDirPath: Path): Seq[FileStatus] = { + fileSystem.listStatus(logDirPath).filter(isEventLogFile) + .sortBy(fs => getSequence(fs.getPath.getName)) + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala new file mode 100644 index 000000000000..5608158edf1e --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config._ + +object EventLogTestHelper { + def getUniqueApplicationId: String = "test-" + System.currentTimeMillis + + /** + * Get a SparkConf with event logging enabled. It doesn't enable rolling event logs, so caller + * should set it manually. + */ + def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None): SparkConf = { + val conf = new SparkConf + conf.set(EVENT_LOG_ENABLED, true) + conf.set(EVENT_LOG_BLOCK_UPDATES, true) + conf.set(EVENT_LOG_TESTING, true) + conf.set(EVENT_LOG_DIR, logDir.toString) + compressionCodec.foreach { codec => + conf.set(EVENT_LOG_COMPRESS, true) + conf.set(EVENT_LOG_COMPRESSION_CODEC, codec) + } + conf.set(EVENT_LOG_STAGE_EXECUTOR_METRICS, true) + conf + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 1d465ba37364..34ccaaf49a98 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -74,8 +74,8 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { appAttemptId: Option[String], inProgress: Boolean, codec: Option[String] = None): File = { - val ip = if (inProgress) EventLoggingListener.IN_PROGRESS else "" - val logUri = EventLoggingListener.getLogPath(testDir.toURI, appId, appAttemptId, codec) + val ip = if (inProgress) EventLogFileWriter.IN_PROGRESS else "" + val logUri = SingleEventLogFileWriter.getLogPath(testDir.toURI, appId, appAttemptId, codec) val logPath = new Path(logUri).toUri.getPath + ip new File(logPath) } @@ -161,10 +161,10 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { class TestFsHistoryProvider extends FsHistoryProvider(createTestConf()) { var mergeApplicationListingCall = 0 override protected def mergeApplicationListing( - fileStatus: FileStatus, + reader: EventLogFileReader, lastSeen: Long, enableSkipToEnd: Boolean): Unit = { - super.mergeApplicationListing(fileStatus, lastSeen, enableSkipToEnd) + super.mergeApplicationListing(reader, lastSeen, enableSkipToEnd) mergeApplicationListingCall += 1 } } @@ -199,13 +199,13 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { ) updateAndCheck(provider) { list => list.size should be (1) - provider.getAttempt("app1", None).logPath should endWith(EventLoggingListener.IN_PROGRESS) + provider.getAttempt("app1", None).logPath should endWith(EventLogFileWriter.IN_PROGRESS) } logFile1.renameTo(newLogFile("app1", None, inProgress = false)) updateAndCheck(provider) { list => list.size should be (1) - provider.getAttempt("app1", None).logPath should not endWith(EventLoggingListener.IN_PROGRESS) + provider.getAttempt("app1", None).logPath should not endWith(EventLogFileWriter.IN_PROGRESS) } } @@ -1161,29 +1161,45 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { when(mockedFs.open(path)).thenReturn(in) when(in.getWrappedStream).thenReturn(dfsIn) when(dfsIn.getFileLength).thenReturn(200) + // FileStatus.getLen is more than logInfo fileSize var fileStatus = new FileStatus(200, false, 0, 0, 0, path) + when(mockedFs.getFileStatus(path)).thenReturn(fileStatus) var logInfo = new LogInfo(path.toString, 0, LogType.EventLogs, Some("appId"), - Some("attemptId"), 100) - assert(mockedProvider.shouldReloadLog(logInfo, fileStatus)) + Some("attemptId"), 100, None, false) + var reader = EventLogFileReader(mockedFs, path) + assert(reader.isDefined) + assert(mockedProvider.shouldReloadLog(logInfo, reader.get)) fileStatus = new FileStatus() fileStatus.setPath(path) + when(mockedFs.getFileStatus(path)).thenReturn(fileStatus) // DFSInputStream.getFileLength is more than logInfo fileSize logInfo = new LogInfo(path.toString, 0, LogType.EventLogs, Some("appId"), - Some("attemptId"), 100) - assert(mockedProvider.shouldReloadLog(logInfo, fileStatus)) + Some("attemptId"), 100, None, false) + reader = EventLogFileReader(mockedFs, path) + assert(reader.isDefined) + assert(mockedProvider.shouldReloadLog(logInfo, reader.get)) + // DFSInputStream.getFileLength is equal to logInfo fileSize logInfo = new LogInfo(path.toString, 0, LogType.EventLogs, Some("appId"), - Some("attemptId"), 200) - assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus)) + Some("attemptId"), 200, None, false) + reader = EventLogFileReader(mockedFs, path) + assert(reader.isDefined) + assert(!mockedProvider.shouldReloadLog(logInfo, reader.get)) + // in.getWrappedStream returns other than DFSInputStream val bin = mock(classOf[BufferedInputStream]) when(in.getWrappedStream).thenReturn(bin) - assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus)) + reader = EventLogFileReader(mockedFs, path) + assert(reader.isDefined) + assert(!mockedProvider.shouldReloadLog(logInfo, reader.get)) + // fs.open throws exception when(mockedFs.open(path)).thenThrow(new IOException("Throwing intentionally")) - assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus)) + reader = EventLogFileReader(mockedFs, path) + assert(reader.isDefined) + assert(!mockedProvider.shouldReloadLog(logInfo, reader.get)) } test("log cleaner with the maximum number of log files") { @@ -1256,7 +1272,11 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { val fstream = new FileOutputStream(file) val cstream = codec.map(_.compressedContinuousOutputStream(fstream)).getOrElse(fstream) val bstream = new BufferedOutputStream(cstream) - EventLoggingListener.initEventLog(bstream, false, null) + + val metadata = SparkListenerLogStart(org.apache.spark.SPARK_VERSION) + val eventJson = JsonProtocol.logStartToJson(metadata) + val metadataJson = compact(eventJson) + "\n" + bstream.write(metadataJson.getBytes(StandardCharsets.UTF_8)) val writer = new OutputStreamWriter(bstream, StandardCharsets.UTF_8) Utils.tryWithSafeFinally { diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index ae55d1915fa4..286924001e92 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import java.io.{File, FileOutputStream, InputStream, IOException} +import java.io.{File, InputStream} import java.util.Arrays import scala.collection.immutable.Map @@ -32,15 +32,15 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.{EventLogFileReader, SingleEventLogFileWriter} +import org.apache.spark.deploy.history.EventLogTestHelper._ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ import org.apache.spark.io._ import org.apache.spark.metrics.{ExecutorMetricType, MetricsSystem} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.{JsonProtocol, Utils} - /** * Test whether EventLoggingListener logs events properly. * @@ -51,8 +51,6 @@ import org.apache.spark.util.{JsonProtocol, Utils} class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfter with Logging { - import EventLoggingListenerSuite._ - private val fileSystem = Utils.getHadoopFileSystem("/", SparkHadoopUtil.get.newConfiguration(new SparkConf())) private var testDir: File = _ @@ -68,40 +66,6 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit Utils.deleteRecursively(testDir) } - test("Verify log file exist") { - // Verify logging directory exists - val conf = getLoggingConf(testDirPath) - val eventLogger = new EventLoggingListener("test", None, testDirPath.toUri(), conf) - eventLogger.start() - - val logPath = new Path(eventLogger.logPath + EventLoggingListener.IN_PROGRESS) - assert(fileSystem.exists(logPath)) - val logStatus = fileSystem.getFileStatus(logPath) - assert(!logStatus.isDirectory) - - // Verify log is renamed after stop() - eventLogger.stop() - assert(!fileSystem.getFileStatus(new Path(eventLogger.logPath)).isDirectory) - } - - test("Basic event logging") { - testEventLogging() - } - - test("spark.eventLog.compression.codec overrides spark.io.compression.codec") { - val conf = new SparkConf - conf.set(EVENT_LOG_COMPRESS, true) - - // The default value is `spark.io.compression.codec`. - val e = new EventLoggingListener("test", None, testDirPath.toUri(), conf) - assert(e.compressionCodecName.contains("lz4")) - - // `spark.eventLog.compression.codec` overrides `spark.io.compression.codec`. - conf.set(EVENT_LOG_COMPRESSION_CODEC, "zstd") - val e2 = new EventLoggingListener("test", None, testDirPath.toUri(), conf) - assert(e2.compressionCodecName.contains("zstd")) - } - test("Basic event logging with compression") { CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => testEventLogging(compressionCodec = Some(CompressionCodec.getShortName(codec))) @@ -131,35 +95,6 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit assert(redactedProps(key) == "*********(redacted)") } - test("Log overwriting") { - val logUri = EventLoggingListener.getLogPath(testDir.toURI, "test", None) - val logPath = new Path(logUri).toUri.getPath - // Create file before writing the event log - new FileOutputStream(new File(logPath)).close() - // Expected IOException, since we haven't enabled log overwrite. - intercept[IOException] { testEventLogging() } - // Try again, but enable overwriting. - testEventLogging(extraConf = Map(EVENT_LOG_OVERWRITE.key -> "true")) - } - - test("Event log name") { - val baseDirUri = Utils.resolveURI("/base-dir") - // without compression - assert(s"${baseDirUri.toString}/app1" === EventLoggingListener.getLogPath( - baseDirUri, "app1", None)) - // with compression - assert(s"${baseDirUri.toString}/app1.lzf" === - EventLoggingListener.getLogPath(baseDirUri, "app1", None, Some("lzf"))) - // illegal characters in app ID - assert(s"${baseDirUri.toString}/a-fine-mind_dollar_bills__1" === - EventLoggingListener.getLogPath(baseDirUri, - "a fine:mind$dollar{bills}.1", None)) - // illegal characters in app ID with compression - assert(s"${baseDirUri.toString}/a-fine-mind_dollar_bills__1.lz4" === - EventLoggingListener.getLogPath(baseDirUri, - "a fine:mind$dollar{bills}.1", None, Some("lz4"))) - } - test("Executor metrics update") { testStageExecutorMetricsEventLogging() } @@ -168,8 +103,6 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit * Actual test logic * * ----------------- */ - import EventLoggingListenerSuite._ - /** * Test basic event logging functionality. * @@ -198,7 +131,8 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit eventLogger.stop() // Verify file contains exactly the two events logged - val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) + val logPath = eventLogger.logWriter.logPath + val logData = EventLogFileReader.openEventLog(new Path(logPath), fileSystem) try { val lines = readLines(logData) val logStart = SparkListenerLogStart(SPARK_VERSION) @@ -226,9 +160,10 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit sc = new SparkContext("local-cluster[2,2,1024]", "test", conf) assert(sc.eventLogger.isDefined) val eventLogger = sc.eventLogger.get - val eventLogPath = eventLogger.logPath + + val eventLogPath = eventLogger.logWriter.logPath val expectedLogDir = testDir.toURI() - assert(eventLogPath === EventLoggingListener.getLogPath( + assert(eventLogPath === SingleEventLogFileWriter.getLogPath( expectedLogDir, sc.applicationId, None, compressionCodec.map(CompressionCodec.getShortName))) // Begin listening for events that trigger asserts @@ -243,7 +178,8 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit eventExistenceListener.assertAllCallbacksInvoked() // Make sure expected events exist in the log file. - val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) + val logData = EventLogFileReader.openEventLog(new Path(eventLogger.logWriter.logPath), + fileSystem) val eventSet = mutable.Set( SparkListenerApplicationStart, SparkListenerBlockManagerAdded, @@ -466,7 +402,8 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // Verify the log file contains the expected events. // Posted events should be logged, except for ExecutorMetricsUpdate events -- these // are consolidated, and the peak values for each stage are logged at stage end. - val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) + val logData = EventLogFileReader.openEventLog(new Path(eventLogger.logWriter.logPath), + fileSystem) try { val lines = readLines(logData) val logStart = SparkListenerLogStart(SPARK_VERSION) @@ -641,24 +578,3 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } } - - -object EventLoggingListenerSuite { - - /** Get a SparkConf with event logging enabled. */ - def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None): SparkConf = { - val conf = new SparkConf - conf.set(EVENT_LOG_ENABLED, true) - conf.set(EVENT_LOG_BLOCK_UPDATES, true) - conf.set(EVENT_LOG_TESTING, true) - conf.set(EVENT_LOG_DIR, logDir.toString) - compressionCodec.foreach { codec => - conf.set(EVENT_LOG_COMPRESS, true) - conf.set(EVENT_LOG_COMPRESSION_CODEC, codec) - } - conf.set(EVENT_LOG_STAGE_EXECUTOR_METRICS, true) - conf - } - - def getUniqueApplicationId: String = "test-" + System.currentTimeMillis -} diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 55e7f5333c67..e6fbf9b09d43 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler import java.io._ -import java.net.URI import java.nio.charset.StandardCharsets import java.util.concurrent.atomic.AtomicInteger @@ -31,6 +30,8 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.EventLogFileReader +import org.apache.spark.deploy.history.EventLogTestHelper._ import org.apache.spark.io.{CompressionCodec, LZ4CompressionCodec} import org.apache.spark.util.{JsonProtocol, JsonProtocolSuite, Utils} @@ -64,7 +65,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp // scalastyle:on println } - val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) + val conf = getLoggingConf(logFilePath) val logData = fileSystem.open(logFilePath) val eventMonster = new EventBufferingListener try { @@ -110,14 +111,14 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp } // Read the compressed .inprogress file and verify only first event was parsed. - val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) + val conf = getLoggingConf(logFilePath) val replayer = new ReplayListenerBus() val eventMonster = new EventBufferingListener replayer.addListener(eventMonster) // Verify the replay returns the events given the input maybe truncated. - val logData = EventLoggingListener.openEventLog(logFilePath, fileSystem) + val logData = EventLogFileReader.openEventLog(logFilePath, fileSystem) Utils.tryWithResource(new EarlyEOFInputStream(logData, buffered.size - 10)) { failingStream => replayer.replay(failingStream, logFilePath.toString, true) @@ -126,7 +127,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp } // Verify the replay throws the EOF exception since the input may not be truncated. - val logData2 = EventLoggingListener.openEventLog(logFilePath, fileSystem) + val logData2 = EventLogFileReader.openEventLog(logFilePath, fileSystem) Utils.tryWithResource(new EarlyEOFInputStream(logData2, buffered.size - 10)) { failingStream2 => intercept[EOFException] { replayer.replay(failingStream2, logFilePath.toString, false) @@ -149,7 +150,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp // scalastyle:on println } - val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) + val conf = getLoggingConf(logFilePath) val logData = fileSystem.open(logFilePath) val eventMonster = new EventBufferingListener try { @@ -195,7 +196,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp val logDirPath = new Path(logDir.toURI) fileSystem.mkdirs(logDirPath) - val conf = EventLoggingListenerSuite.getLoggingConf(logDirPath, codecName) + val conf = getLoggingConf(logDirPath, codecName) sc = new SparkContext("local-cluster[2,1,1024]", "Test replay", conf) // Run a few jobs @@ -212,7 +213,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp assert(!eventLog.isDirectory) // Replay events - val logData = EventLoggingListener.openEventLog(eventLog.getPath(), fileSystem) + val logData = EventLogFileReader.openEventLog(eventLog.getPath(), fileSystem) val eventMonster = new EventBufferingListener try { val replayer = new ReplayListenerBus() diff --git a/docs/configuration.md b/docs/configuration.md index 729b1ba7ed2c..3420007082bf 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1008,6 +1008,21 @@ Apart from these, the following properties are also available, and may be useful Buffer size to use when writing to output streams, in KiB unless otherwise specified. + + spark.eventLog.logRolling.enabled + false + + Whether rolling over event log files is enabled. If set to true, it cuts down each event + log file to be configured size. + + + + spark.eventLog.logRolling.maxFileSize + 128m + + The max size of event log file to be rolled over. + + spark.ui.dagGraph.retainedRootRDDs Int.MaxValue From 90698e8a446bd2754b9bf3a2c49eb9aadf17ab06 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 25 Sep 2019 21:23:16 +0900 Subject: [PATCH 02/15] Reflect review comments (low hanging first) --- .../deploy/history/EventLogFileWriters.scala | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala index 2d9cd0ce208a..cde11f81efad 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -299,7 +299,7 @@ class RollingEventLogFilesWriter( // seq and event log path will be updated soon in rollEventLogFile, which `start` will call private var index: Long = 0L - private var currentEventLogFilePath: Path = logDirForAppPath + private var currentEventLogFilePath: Path = _ override def start(): Unit = { requireLogBaseDirAsDirectory() @@ -368,8 +368,12 @@ class RollingEventLogFilesWriter( } object RollingEventLogFilesWriter { + private val EVENT_LOG_DIR_NAME_PREFIX = "eventlog_v2_" + private val EVENT_LOG_FILE_NAME_PREFIX = "events_" + private val APPSTATUS_FILE_NAME_PREFIX = "appstatus_" + def getAppEventLogDirPath(logBaseDir: URI, appId: String, appAttemptId: Option[String]): Path = - new Path(new Path(logBaseDir), "eventlog_v2_" + + new Path(new Path(logBaseDir), EVENT_LOG_DIR_NAME_PREFIX + EventLogFileWriter.nameForAppAndAttempt(appId, appAttemptId)) def getAppStatusFilePath( @@ -377,7 +381,8 @@ object RollingEventLogFilesWriter { appId: String, appAttemptId: Option[String], inProgress: Boolean): Path = { - val base = "appstatus_" + EventLogFileWriter.nameForAppAndAttempt(appId, appAttemptId) + val base = APPSTATUS_FILE_NAME_PREFIX + + EventLogFileWriter.nameForAppAndAttempt(appId, appAttemptId) val name = if (inProgress) base + EventLogFileWriter.IN_PROGRESS else base new Path(appLogDir, name) } @@ -388,13 +393,14 @@ object RollingEventLogFilesWriter { appAttemptId: Option[String], index: Long, codecName: Option[String]): Path = { - val base = s"events_${index}_" + EventLogFileWriter.nameForAppAndAttempt(appId, appAttemptId) + val base = s"${EVENT_LOG_FILE_NAME_PREFIX}_${index}_" + + EventLogFileWriter.nameForAppAndAttempt(appId, appAttemptId) val codec = codecName.map("." + _).getOrElse("") new Path(appLogDir, base + codec) } def isEventLogDir(status: FileStatus): Boolean = { - status.isDirectory && status.getPath.getName.startsWith("eventlog_v2_") + status.isDirectory && status.getPath.getName.startsWith(EVENT_LOG_DIR_NAME_PREFIX) } def isEventLogFile(status: FileStatus): Boolean = { @@ -402,7 +408,7 @@ object RollingEventLogFilesWriter { } def isEventLogFile(path: Path): Boolean = { - path.getName.startsWith("events_") + path.getName.startsWith(EVENT_LOG_FILE_NAME_PREFIX) } def isAppStatusFile(status: FileStatus): Boolean = { @@ -410,12 +416,12 @@ object RollingEventLogFilesWriter { } def isAppStatusFile(path: Path): Boolean = { - path.getName.startsWith("appstatus") + path.getName.startsWith(APPSTATUS_FILE_NAME_PREFIX) } def getSequence(eventLogFileName: String): Long = { - require(eventLogFileName.startsWith("events_"), "Not a event log file!") - val seq = eventLogFileName.stripPrefix("events_").split("_")(0) + require(eventLogFileName.startsWith(EVENT_LOG_FILE_NAME_PREFIX), "Not a event log file!") + val seq = eventLogFileName.stripPrefix(EVENT_LOG_DIR_NAME_PREFIX).split("_")(0) seq.toLong } } From e0abbaecade12252d65b9c95de04146fd2741b97 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 25 Sep 2019 21:45:57 +0900 Subject: [PATCH 03/15] Reflect review comment, but a bit differently --- .../deploy/history/EventLogFileWriters.scala | 68 +++++++++---------- 1 file changed, 32 insertions(+), 36 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala index cde11f81efad..b2af00aacdbf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -70,13 +70,17 @@ abstract class EventLogFileWriter( CompressionCodec.getShortName(c.getClass.getName) } + // Only defined if the file system scheme is not local + protected var hadoopDataStream: Option[FSDataOutputStream] = None + protected var writer: Option[PrintWriter] = None + protected def requireLogBaseDirAsDirectory(): Unit = { if (!fileSystem.getFileStatus(new Path(logBaseDir)).isDirectory) { throw new IllegalArgumentException(s"Log directory $logBaseDir is not a directory.") } } - protected def initLogFile(path: Path): (Option[FSDataOutputStream], OutputStream) = { + protected def initLogFile(path: Path, fnSetupWriter: OutputStream => PrintWriter): Unit = { if (shouldOverwrite && fileSystem.delete(path, true)) { logWarning(s"Event log $path already exists. Overwriting...") } @@ -85,7 +89,6 @@ abstract class EventLogFileWriter( val isDefaultLocal = defaultFs == null || defaultFs == "file" val uri = path.toUri - var hadoopDataStream: Option[FSDataOutputStream] = None /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). * Therefore, for local files, use FileOutputStream instead. */ val dstream = @@ -103,7 +106,7 @@ abstract class EventLogFileWriter( val bstream = new BufferedOutputStream(cstream, outputBufferSize) fileSystem.setPermission(path, EventLogFileWriter.LOG_FILE_PERMISSIONS) logInfo(s"Logging events to $path") - (hadoopDataStream, bstream) + writer = Some(fnSetupWriter(bstream)) } catch { case e: Exception => dstream.close() @@ -111,6 +114,20 @@ abstract class EventLogFileWriter( } } + protected def writeJson(json: String, flushLogger: Boolean = false): Unit = { + // scalastyle:off println + writer.foreach(_.println(json)) + // scalastyle:on println + if (flushLogger) { + writer.foreach(_.flush()) + hadoopDataStream.foreach(_.hflush()) + } + } + + protected def closeWriter(): Unit = { + writer.foreach(_.close()) + } + protected def renameFile(src: Path, dest: Path, overwrite: Boolean): Unit = { if (fileSystem.exists(dest)) { if (overwrite) { @@ -197,27 +214,15 @@ class SingleEventLogFileWriter( private val inProgressPath = logPath + EventLogFileWriter.IN_PROGRESS - // Only defined if the file system scheme is not local - private var hadoopDataStream: Option[FSDataOutputStream] = None - - private var writer: Option[PrintWriter] = None - override def start(): Unit = { requireLogBaseDirAsDirectory() - val (hadoopStream, outputStream) = initLogFile(new Path(inProgressPath)) - hadoopDataStream = hadoopStream - writer = Some(new PrintWriter(new OutputStreamWriter(outputStream, StandardCharsets.UTF_8))) + initLogFile(new Path(inProgressPath), + ostream => new PrintWriter(new OutputStreamWriter(ostream, StandardCharsets.UTF_8))) } override def writeEvent(eventJson: String, flushLogger: Boolean = false): Unit = { - // scalastyle:off println - writer.foreach(_.println(eventJson)) - // scalastyle:on println - if (flushLogger) { - writer.foreach(_.flush()) - hadoopDataStream.foreach(_.hflush()) - } + writeJson(eventJson, flushLogger) } /** @@ -225,7 +230,7 @@ class SingleEventLogFileWriter( * ".inprogress" suffix. */ override def stop(): Unit = { - writer.foreach(_.close()) + closeWriter() renameFile(new Path(inProgressPath), new Path(logPath), shouldOverwrite) } } @@ -292,10 +297,7 @@ class RollingEventLogFilesWriter( private val logDirForAppPath = getAppEventLogDirPath(logBaseDir, appId, appAttemptId) - // Only defined if the file system scheme is not local - private var hadoopDataStream: Option[FSDataOutputStream] = None private var countingOutputStream: Option[CountingOutputStream] = None - private var writer: Option[PrintWriter] = None // seq and event log path will be updated soon in rollEventLogFile, which `start` will call private var index: Long = 0L @@ -325,31 +327,25 @@ class RollingEventLogFilesWriter( } } - // scalastyle:off println - writer.foreach(_.println(eventJson)) - // scalastyle:on println - if (flushLogger) { - writer.foreach(_.flush()) - hadoopDataStream.foreach(_.hflush()) - } + writeJson(eventJson, flushLogger) } private def rollEventLogFile(): Unit = { - writer.foreach(_.close()) + closeWriter() index += 1 currentEventLogFilePath = getEventLogFilePath(logDirForAppPath, appId, appAttemptId, index, compressionCodecName) - val (hadoopStream, outputStream) = initLogFile(currentEventLogFilePath) - hadoopDataStream = hadoopStream - countingOutputStream = Some(new CountingOutputStream(outputStream)) - writer = Some(new PrintWriter( - new OutputStreamWriter(countingOutputStream.get, StandardCharsets.UTF_8))) + initLogFile(currentEventLogFilePath, ostream => { + countingOutputStream = Some(new CountingOutputStream(ostream)) + new PrintWriter( + new OutputStreamWriter(countingOutputStream.get, StandardCharsets.UTF_8)) + }) } override def stop(): Unit = { - writer.foreach(_.close()) + closeWriter() val appStatusPathIncomplete = getAppStatusFilePath(logDirForAppPath, appId, appAttemptId, inProgress = true) val appStatusPathComplete = getAppStatusFilePath(logDirForAppPath, appId, appAttemptId, From f1087e893a81ec56acbd3a030dff14b5d7e2424e Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 26 Sep 2019 04:58:36 +0900 Subject: [PATCH 04/15] Fix silly mistakes to fix UTs --- .../org/apache/spark/deploy/history/EventLogFileWriters.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala index b2af00aacdbf..240b53cdd230 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -389,7 +389,7 @@ object RollingEventLogFilesWriter { appAttemptId: Option[String], index: Long, codecName: Option[String]): Path = { - val base = s"${EVENT_LOG_FILE_NAME_PREFIX}_${index}_" + + val base = s"${EVENT_LOG_FILE_NAME_PREFIX}${index}_" + EventLogFileWriter.nameForAppAndAttempt(appId, appAttemptId) val codec = codecName.map("." + _).getOrElse("") new Path(appLogDir, base + codec) @@ -417,7 +417,7 @@ object RollingEventLogFilesWriter { def getSequence(eventLogFileName: String): Long = { require(eventLogFileName.startsWith(EVENT_LOG_FILE_NAME_PREFIX), "Not a event log file!") - val seq = eventLogFileName.stripPrefix(EVENT_LOG_DIR_NAME_PREFIX).split("_")(0) + val seq = eventLogFileName.stripPrefix(EVENT_LOG_FILE_NAME_PREFIX).split("_")(0) seq.toLong } } From 064ea7b2f86652fece5610fd6d5a6607e320a8b3 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 26 Sep 2019 11:05:37 +0900 Subject: [PATCH 05/15] Reflect review comments --- .../deploy/history/EventLogFileReaders.scala | 24 +++++++------- .../deploy/history/EventLogFileWriters.scala | 32 +++++++------------ .../deploy/history/FsHistoryProvider.scala | 23 +++++++------ .../spark/internal/config/package.scala | 5 +-- .../history/EventLogFileReadersSuite.scala | 3 +- 5 files changed, 39 insertions(+), 48 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala index 7e46d2a425cd..85eec6c47e73 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala @@ -18,10 +18,9 @@ package org.apache.spark.deploy.history import java.io.{BufferedInputStream, InputStream} +import java.util.concurrent.ConcurrentHashMap import java.util.zip.{ZipEntry, ZipOutputStream} -import scala.collection.mutable - import com.google.common.io.ByteStreams import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.hdfs.DFSInputStream @@ -96,13 +95,13 @@ abstract class EventLogFileReader( object EventLogFileReader { // A cache for compression codecs to avoid creating the same codec many times - private val codecMap = mutable.HashMap.empty[String, CompressionCodec] + private val codecMap = new ConcurrentHashMap[String, CompressionCodec]() def apply( fs: FileSystem, path: Path, - lastSequenceNumber: Option[Long]): EventLogFileReader = { - lastSequenceNumber match { + lastIndex: Option[Long]): EventLogFileReader = { + lastIndex match { case Some(_) => new RollingEventLogFilesFileReader(fs, path) case None => new SingleFileEventLogFileReader(fs, path) } @@ -131,7 +130,7 @@ object EventLogFileReader { val in = new BufferedInputStream(fs.open(log)) try { val codec = codecName(log).map { c => - codecMap.getOrElseUpdate(c, CompressionCodec.createCodec(new SparkConf, c)) + codecMap.computeIfAbsent(c, CompressionCodec.createCodec(new SparkConf, _)) } codec.map(_.compressedContinuousInputStream(in)).getOrElse(in) } catch { @@ -158,7 +157,6 @@ object EventLogFileReader { class SingleFileEventLogFileReader( fs: FileSystem, path: Path) extends EventLogFileReader(fs, path) { - // TODO: get stats with constructor and only call if it's needed? private lazy val status = fileSystem.getFileStatus(rootPath) override def lastIndex: Option[Long] = None @@ -177,8 +175,9 @@ class SingleFileEventLogFileReader( override def modificationTime: Long = status.getModificationTime - override def zipEventLogFiles(zipStream: ZipOutputStream): Unit = + override def zipEventLogFiles(zipStream: ZipOutputStream): Unit = { addFileAsZipEntry(zipStream, rootPath, rootPath.getName) + } override def listEventLogFiles: Seq[FileStatus] = Seq(status) @@ -202,7 +201,7 @@ class RollingEventLogFilesFileReader( override def lastIndex: Option[Long] = { val maxSeq = files.filter(isEventLogFile) - .map { stats => getSequence(stats.getPath.getName) } + .map { status => getSequence(status.getPath.getName) } .max Some(maxSeq) } @@ -235,13 +234,14 @@ class RollingEventLogFilesFileReader( override def listEventLogFiles: Seq[FileStatus] = eventLogFiles - override def compressionCodec: Option[String] = EventLogFileWriter.codecName( - eventLogFiles.head.getPath) + override def compressionCodec: Option[String] = { + EventLogFileWriter.codecName(eventLogFiles.head.getPath) + } override def totalSize: Long = eventLogFiles.map(_.getLen).sum private def eventLogFiles: Seq[FileStatus] = { - files.filter(isEventLogFile).sortBy { stats => getSequence(stats.getPath.getName) } + files.filter(isEventLogFile).sortBy { status => getSequence(status.getPath.getName) } } private def lastEventLogFile: FileStatus = eventLogFiles.reverse.head diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala index 240b53cdd230..3a28815b5168 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -56,7 +56,6 @@ abstract class EventLogFileWriter( protected val shouldCompress = sparkConf.get(EVENT_LOG_COMPRESS) protected val shouldOverwrite = sparkConf.get(EVENT_LOG_OVERWRITE) - protected val shouldAllowECLogs = sparkConf.get(EVENT_LOG_ALLOW_EC) protected val outputBufferSize = sparkConf.get(EVENT_LOG_OUTPUT_BUFFER_SIZE).toInt protected val fileSystem = Utils.getHadoopFileSystem(logBaseDir, hadoopConf) protected val compressionCodec = @@ -80,7 +79,7 @@ abstract class EventLogFileWriter( } } - protected def initLogFile(path: Path, fnSetupWriter: OutputStream => PrintWriter): Unit = { + protected def initLogFile(path: Path)(fnSetupWriter: OutputStream => PrintWriter): Unit = { if (shouldOverwrite && fileSystem.delete(path, true)) { logWarning(s"Event log $path already exists. Overwriting...") } @@ -141,7 +140,7 @@ abstract class EventLogFileWriter( } fileSystem.rename(src, dest) // touch file to ensure modtime is current across those filesystems where rename() - // does not set it, -and which support setTimes(); it's a no-op on most object stores + // does not set it but support setTimes() instead; it's a no-op on most object stores try { fileSystem.setTimes(dest, System.currentTimeMillis(), -1) } catch { @@ -217,8 +216,9 @@ class SingleEventLogFileWriter( override def start(): Unit = { requireLogBaseDirAsDirectory() - initLogFile(new Path(inProgressPath), - ostream => new PrintWriter(new OutputStreamWriter(ostream, StandardCharsets.UTF_8))) + initLogFile(new Path(inProgressPath)) { os => + new PrintWriter(new OutputStreamWriter(os, StandardCharsets.UTF_8)) + } } override def writeEvent(eventJson: String, flushLogger: Boolean = false): Unit = { @@ -293,7 +293,7 @@ class RollingEventLogFilesWriter( import RollingEventLogFilesWriter._ - private val eventFileMaxLengthKiB = sparkConf.get(EVENT_LOG_ROLLED_EVENT_LOG_MAX_FILE_SIZE) + private val eventFileMaxLength = sparkConf.get(EVENT_LOG_ROLLED_EVENT_LOG_MAX_FILE_SIZE) private val logDirForAppPath = getAppEventLogDirPath(logBaseDir, appId, appAttemptId) @@ -322,7 +322,7 @@ class RollingEventLogFilesWriter( override def writeEvent(eventJson: String, flushLogger: Boolean = false): Unit = { writer.foreach { w => val currentLen = countingOutputStream.get.getBytesWritten - if (currentLen + eventJson.length > eventFileMaxLengthKiB * 1024) { + if (currentLen + eventJson.length > eventFileMaxLength * 1024 * 1024) { rollEventLogFile() } } @@ -337,11 +337,11 @@ class RollingEventLogFilesWriter( currentEventLogFilePath = getEventLogFilePath(logDirForAppPath, appId, appAttemptId, index, compressionCodecName) - initLogFile(currentEventLogFilePath, ostream => { - countingOutputStream = Some(new CountingOutputStream(ostream)) + initLogFile(currentEventLogFilePath) { os => + countingOutputStream = Some(new CountingOutputStream(os)) new PrintWriter( new OutputStreamWriter(countingOutputStream.get, StandardCharsets.UTF_8)) - }) + } } override def stop(): Unit = { @@ -400,19 +400,11 @@ object RollingEventLogFilesWriter { } def isEventLogFile(status: FileStatus): Boolean = { - status.isFile && isEventLogFile(status.getPath) - } - - def isEventLogFile(path: Path): Boolean = { - path.getName.startsWith(EVENT_LOG_FILE_NAME_PREFIX) + status.isFile && status.getPath.getName.startsWith(EVENT_LOG_FILE_NAME_PREFIX) } def isAppStatusFile(status: FileStatus): Boolean = { - status.isFile && isAppStatusFile(status.getPath) - } - - def isAppStatusFile(path: Path): Boolean = { - path.getName.startsWith(APPSTATUS_FILE_NAME_PREFIX) + status.isFile && status.getPath.getName.startsWith(APPSTATUS_FILE_NAME_PREFIX) } def getSequence(eventLogFileName: String): Long = { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 115eb19751bd..bad96e36a98d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -451,7 +451,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // case the code below decides we don't need to parse the log. listing.write(info.copy(lastProcessed = newLastScanTime, fileSize = reader.fileSizeForLastIndex, - lastSequenceNum = reader.lastIndex, + lastIndex = reader.lastIndex, isComplete = reader.completed)) } @@ -472,7 +472,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) attempt.info.copy(lastUpdated = new Date(newLastScanTime)), attempt.logPath, attempt.fileSize, - attempt.lastSequenceNum, + attempt.lastIndex, attempt.adminAcls, attempt.viewAcls, attempt.adminAclsGroups, @@ -580,10 +580,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) if (info.isComplete != reader.completed) { true } else { - var result = if (info.lastSequenceNum.isDefined) { + var result = if (info.lastIndex.isDefined) { require(reader.lastIndex.isDefined) - info.lastSequenceNum.get < reader.lastIndex.get || - info.fileSize < reader.fileSizeForLastIndex + info.lastIndex.get < reader.lastIndex.get || info.fileSize < reader.fileSizeForLastIndex } else { info.fileSize < reader.fileSizeForLastIndex } @@ -657,7 +656,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) .getOrElse(app.attempts) .foreach { attempt => val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), - attempt.lastSequenceNum) + attempt.lastIndex) reader.zipEventLogFiles(zipStream) } } finally { @@ -1070,7 +1069,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // load, so the event log needs to be replayed. val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), - attempt.lastSequenceNum) + attempt.lastIndex) val isCompressed = reader.compressionCodec.isDefined logInfo(s"Leasing disk manager space for app $appId / ${attempt.info.attemptId}...") val lease = dm.lease(reader.totalSize, isCompressed) @@ -1091,7 +1090,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private def createInMemoryStore(attempt: AttemptInfoWrapper): KVStore = { val store = new InMemoryStore() val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), - attempt.lastSequenceNum) + attempt.lastIndex) rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime()) store } @@ -1163,14 +1162,14 @@ private[history] case class LogInfo( appId: Option[String], attemptId: Option[String], fileSize: Long, - lastSequenceNum: Option[Long], + lastIndex: Option[Long], isComplete: Boolean) private[history] class AttemptInfoWrapper( val info: ApplicationAttemptInfo, val logPath: String, val fileSize: Long, - val lastSequenceNum: Option[Long], + val lastIndex: Option[Long], val adminAcls: Option[String], val viewAcls: Option[String], val adminAclsGroups: Option[String], @@ -1285,7 +1284,7 @@ private[history] class AppListingListener( } - private class MutableAttemptInfo(logPath: String, fileSize: Long, lastSequence: Option[Long]) { + private class MutableAttemptInfo(logPath: String, fileSize: Long, lastIndex: Option[Long]) { var attemptId: Option[String] = None var startTime = new Date(-1) var endTime = new Date(-1) @@ -1314,7 +1313,7 @@ private[history] class AppListingListener( apiInfo, logPath, fileSize, - lastSequence, + lastIndex, adminAcls, viewAcls, adminAclsGroups, diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index f6c21a6d6de9..39ffe5229a46 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -182,12 +182,13 @@ package object config { private[spark] val EVENT_LOG_ENABLE_ROLLING = ConfigBuilder("spark.eventLog.logRolling.enabled") .doc("Whether rolling over event log files is enabled.") - .booleanConf.createWithDefault(false) + .booleanConf + .createWithDefault(false) private[spark] val EVENT_LOG_ROLLED_EVENT_LOG_MAX_FILE_SIZE = ConfigBuilder("spark.eventLog.logRolling.maxFileSize") .doc("The max size of event log file to be rolled over.") - .bytesConf(ByteUnit.KiB) + .bytesConf(ByteUnit.BYTE) .createWithDefaultString("128m") private[spark] val EXECUTOR_ID = diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala index 5bec9b49209f..869f8c7b9b22 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala @@ -42,13 +42,12 @@ abstract class EventLogFileReadersSuite extends SparkFunSuite with LocalSparkCon with BeforeAndAfter with Logging { protected val fileSystem = Utils.getHadoopFileSystem("/", - SparkHadoopUtil.get.newConfiguration(new SparkConf())) + SparkHadoopUtil.get.conf) protected var testDir: File = _ protected var testDirPath: Path = _ before { testDir = Utils.createTempDir(namePrefix = s"event log") - testDir.deleteOnExit() testDirPath = new Path(testDir.getAbsolutePath()) } From 9c72cc8bd2af5a630480773d08eccc0079b21f33 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 26 Sep 2019 14:02:11 +0900 Subject: [PATCH 06/15] Fix silly mistake --- .../org/apache/spark/deploy/history/EventLogFileWriters.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala index 3a28815b5168..d90cf11446d5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -322,7 +322,7 @@ class RollingEventLogFilesWriter( override def writeEvent(eventJson: String, flushLogger: Boolean = false): Unit = { writer.foreach { w => val currentLen = countingOutputStream.get.getBytesWritten - if (currentLen + eventJson.length > eventFileMaxLength * 1024 * 1024) { + if (currentLen + eventJson.length > eventFileMaxLength) { rollEventLogFile() } } From 63990e0ac62d4c52cee6b7305cdab455e5f72b6c Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 27 Sep 2019 05:15:34 +0900 Subject: [PATCH 07/15] Don't test functionality: reading "not yet closed" event log --- .../history/EventLogFileWritersSuite.scala | 55 +++++-------------- 1 file changed, 14 insertions(+), 41 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala index 17755e1d546b..7d4b9ec86838 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala @@ -96,13 +96,9 @@ abstract class EventLogFileWritersSuite extends SparkFunSuite with LocalSparkCon val dummyData = Seq("dummy1", "dummy2", "dummy3") dummyData.foreach(writer.writeEvent(_, flushLogger = true)) - verifyWriteEventLogFile(appId, attemptId, testDirPath.toUri, codecShortName, - isCompleted = false, dummyData) - writer.stop() - verifyWriteEventLogFile(appId, attemptId, testDirPath.toUri, codecShortName, - isCompleted = true, dummyData) + verifyWriteEventLogFile(appId, attemptId, testDirPath.toUri, codecShortName, dummyData) } } @@ -140,25 +136,16 @@ abstract class EventLogFileWritersSuite extends SparkFunSuite with LocalSparkCon sparkConf: SparkConf, hadoopConf: Configuration): EventLogFileWriter + /** + * This should be called with "closed" event log file; No guarantee on reading event log file + * which is being written, especially the file is compressed. SHS also does the best it can. + */ protected def verifyWriteEventLogFile( appId: String, appAttemptId : Option[String], logBaseDir: URI, compressionCodecShortName: Option[String], - isCompleted: Boolean, expectedLines: Seq[String] = Seq.empty): Unit - - protected def skipVerifyEventLogFile( - compressionCodecShortName: Option[String], - isCompleted: Boolean): Boolean = { - // Spark initializes LZ4BlockOutputStream with syncFlush=false, so we can't force - // pending bytes to be flushed. It's only guaranteed when stream is closed, so - // we only check for lz4 when isCompleted = true. - - // zstd seems to have issue in reading while write stream is in progress of writing - !isCompleted && - (compressionCodecShortName.contains("lz4") || compressionCodecShortName.contains("zstd")) - } } class SingleEventLogFileWriterSuite extends EventLogFileWritersSuite { @@ -223,22 +210,14 @@ class SingleEventLogFileWriterSuite extends EventLogFileWritersSuite { appAttemptId: Option[String], logBaseDir: URI, compressionCodecShortName: Option[String], - isCompleted: Boolean, expectedLines: Seq[String]): Unit = { // read single event log file val logPath = SingleEventLogFileWriter.getLogPath(logBaseDir, appId, appAttemptId, compressionCodecShortName) - val finalLogPath = if (!isCompleted) { - new Path(logPath + EventLogFileWriter.IN_PROGRESS) - } else { - new Path(logPath) - } - + val finalLogPath = new Path(logPath) assert(fileSystem.exists(finalLogPath) && fileSystem.isFile(finalLogPath)) - if (!skipVerifyEventLogFile(compressionCodecShortName, isCompleted)) { - assert(expectedLines === readLinesFromEventLogFile(finalLogPath, fileSystem)) - } + assert(expectedLines === readLinesFromEventLogFile(finalLogPath, fileSystem)) } } @@ -346,16 +325,13 @@ class RollingEventLogFilesWriterSuite extends EventLogFileWritersSuite { val eventLogFiles = listEventLogFiles(logDirPath) assertEventLogFilesSequence(eventLogFiles, 3, 1024 * 1024) - verifyWriteEventLogFile(appId, attemptId, testDirPath.toUri, - codecShortName, isCompleted = false, expectedLines) - writer.stop() val eventLogFiles2 = listEventLogFiles(logDirPath) assertEventLogFilesSequence(eventLogFiles2, 3, 1024 * 1024) verifyWriteEventLogFile(appId, attemptId, testDirPath.toUri, - codecShortName, isCompleted = true, expectedLines) + codecShortName, expectedLines) } } @@ -373,24 +349,21 @@ class RollingEventLogFilesWriterSuite extends EventLogFileWritersSuite { appAttemptId: Option[String], logBaseDir: URI, compressionCodecShortName: Option[String], - isCompleted: Boolean, expectedLines: Seq[String]): Unit = { val logDirPath = getAppEventLogDirPath(logBaseDir, appId, appAttemptId) assert(fileSystem.exists(logDirPath) && fileSystem.isDirectory(logDirPath)) - val appStatusFile = getAppStatusFilePath(logDirPath, appId, appAttemptId, !isCompleted) + val appStatusFile = getAppStatusFilePath(logDirPath, appId, appAttemptId, inProgress = false) assert(fileSystem.exists(appStatusFile) && fileSystem.isFile(appStatusFile)) val eventLogFiles = listEventLogFiles(logDirPath) - if (!skipVerifyEventLogFile(compressionCodecShortName, isCompleted)) { - val allLines = mutable.ArrayBuffer[String]() - eventLogFiles.foreach { file => - allLines.appendAll(readLinesFromEventLogFile(file.getPath, fileSystem)) - } - - assert(expectedLines === allLines) + val allLines = mutable.ArrayBuffer[String]() + eventLogFiles.foreach { file => + allLines.appendAll(readLinesFromEventLogFile(file.getPath, fileSystem)) } + + assert(expectedLines === allLines) } private def listEventLogFiles(logDirPath: Path): Seq[FileStatus] = { From 79754be3c2a04fd0462563a18b60aed3c05380e9 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 27 Sep 2019 05:40:12 +0900 Subject: [PATCH 08/15] Reflect review comments --- .../deploy/history/EventLogFileReaders.scala | 28 +++++++++++++------ .../deploy/history/EventLogFileWriters.scala | 2 +- .../deploy/history/FsHistoryProvider.scala | 2 +- .../history/EventLogFileReadersSuite.scala | 4 +-- .../history/EventLogFileWritersSuite.scala | 4 +-- 5 files changed, 26 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala index 85eec6c47e73..afc520e54d07 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala @@ -73,7 +73,7 @@ abstract class EventLogFileReader( */ def fileSizeForLastIndexForDFS: Option[Long] - /** Returns the modification time for the last sequence number of event log files. */ + /** Returns the modification time for the last index of event log files. */ def modificationTime: Long /** @@ -153,7 +153,14 @@ object EventLogFileReader { } } -/** The reader which will read the information of single event log file. */ +/** + * The reader which will read the information of single event log file. + * + * This reader gets the status of event log file only once when required; + * It may not give "live" status of file that could be changing concurrently, and + * FileNotFoundException could occur if the log file is renamed before getting the + * status of log file. + */ class SingleFileEventLogFileReader( fs: FileSystem, path: Path) extends EventLogFileReader(fs, path) { @@ -186,7 +193,12 @@ class SingleFileEventLogFileReader( override def totalSize: Long = fileSizeForLastIndex } -/** The reader which will read the information of rolled multiple event log files. */ +/** + * The reader which will read the information of rolled multiple event log files. + * + * This reader lists the files only once; if caller would like to play with updated list, + * it needs to create another reader instance. + */ class RollingEventLogFilesFileReader( fs: FileSystem, path: Path) extends EventLogFileReader(fs, path) { @@ -199,9 +211,11 @@ class RollingEventLogFilesFileReader( ret } + private lazy val appStatusFile = files.find(isAppStatusFile).get + override def lastIndex: Option[Long] = { val maxSeq = files.filter(isEventLogFile) - .map { status => getSequence(status.getPath.getName) } + .map { status => getIndex(status.getPath.getName) } .max Some(maxSeq) } @@ -209,9 +223,7 @@ class RollingEventLogFilesFileReader( override def fileSizeForLastIndex: Long = lastEventLogFile.getLen override def completed: Boolean = { - val appStatsFile = files.find(isAppStatusFile) - require(appStatsFile.isDefined) - appStatsFile.exists(!_.getPath.getName.endsWith(EventLogFileWriter.IN_PROGRESS)) + !appStatusFile.getPath.getName.endsWith(EventLogFileWriter.IN_PROGRESS) } override def fileSizeForLastIndexForDFS: Option[Long] = { @@ -241,7 +253,7 @@ class RollingEventLogFilesFileReader( override def totalSize: Long = eventLogFiles.map(_.getLen).sum private def eventLogFiles: Seq[FileStatus] = { - files.filter(isEventLogFile).sortBy { status => getSequence(status.getPath.getName) } + files.filter(isEventLogFile).sortBy { status => getIndex(status.getPath.getName) } } private def lastEventLogFile: FileStatus = eventLogFiles.reverse.head diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala index d90cf11446d5..0dc55a49b5d2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -407,7 +407,7 @@ object RollingEventLogFilesWriter { status.isFile && status.getPath.getName.startsWith(APPSTATUS_FILE_NAME_PREFIX) } - def getSequence(eventLogFileName: String): Long = { + def getIndex(eventLogFileName: String): Long = { require(eventLogFileName.startsWith(EVENT_LOG_FILE_NAME_PREFIX), "Not a event log file!") val seq = eventLogFileName.stripPrefix(EVENT_LOG_FILE_NAME_PREFIX).split("_")(0) seq.toLong diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index bad96e36a98d..06887f6c89fb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -759,7 +759,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // the file is really gone. // The logic is only valid for single event log, as root path doesn't change for // rolled event logs. - if (appCompleted && reader.lastIndex.isDefined) { + if (appCompleted && reader.lastIndex.isEmpty) { val inProgressLog = logPath.toString() + EventLogFileWriter.IN_PROGRESS try { // Fetch the entry first to avoid an RPC when it's already removed. diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala index 869f8c7b9b22..bae22dec1e0c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala @@ -297,13 +297,13 @@ class RollingEventLogFilesReaderSuite extends EventLogFileReadersSuite { assert(stats.isDirectory) val statsInDir = fileSystem.listStatus(logPath) - val eventFiles = statsInDir.filter(isEventLogFile).sortBy(s => getSequence(s.getPath.getName)) + val eventFiles = statsInDir.filter(isEventLogFile).sortBy(s => getIndex(s.getPath.getName)) assert(eventFiles.nonEmpty) val lastEventFile = eventFiles.last val allLen = eventFiles.map(_.getLen).sum assert(reader.rootPath === fileSystem.makeQualified(logPath)) - assert(reader.lastIndex === Some(getSequence(lastEventFile.getPath.getName))) + assert(reader.lastIndex === Some(getIndex(lastEventFile.getPath.getName))) assert(reader.fileSizeForLastIndex === lastEventFile.getLen) assert(reader.completed === isCompleted) assert(reader.modificationTime === lastEventFile.getModificationTime) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala index 7d4b9ec86838..dbc961b46719 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala @@ -294,7 +294,7 @@ class RollingEventLogFilesWriterSuite extends EventLogFileWritersSuite { expectedMaxSizeBytes: Long): Unit = { assert(eventLogFiles.forall(f => f.getLen < expectedMaxSizeBytes)) assert((1 to expectedLastSequence) === - eventLogFiles.map(f => getSequence(f.getPath.getName))) + eventLogFiles.map(f => getIndex(f.getPath.getName))) } val appId = getUniqueApplicationId @@ -368,6 +368,6 @@ class RollingEventLogFilesWriterSuite extends EventLogFileWritersSuite { private def listEventLogFiles(logDirPath: Path): Seq[FileStatus] = { fileSystem.listStatus(logDirPath).filter(isEventLogFile) - .sortBy(fs => getSequence(fs.getPath.getName)) + .sortBy(fs => getIndex(fs.getPath.getName)) } } From ed7986455ebddbfce0622fa78acaf47c79b3122b Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 27 Sep 2019 12:31:36 +0900 Subject: [PATCH 09/15] Reflect review comment --- .../org/apache/spark/deploy/history/EventLogFileWriters.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala index 0dc55a49b5d2..9870b3f83834 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -157,7 +157,7 @@ abstract class EventLogFileWriter( /** stops writer - indicating the application has been completed */ def stop(): Unit - /** returns representative path of log */ + /** returns representative path of log. for tests only. */ def logPath: String } From 27946d13ec9f552a63eb4c6d192f9a98ce2a5349 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 4 Oct 2019 07:53:45 +0900 Subject: [PATCH 10/15] Reflect review comments --- .../deploy/history/EventLogFileReaders.scala | 2 +- .../deploy/history/EventLogFileWriters.scala | 18 ++-- .../spark/internal/config/package.scala | 3 +- .../scheduler/EventLoggingListener.scala | 2 +- .../history/EventLogFileReadersSuite.scala | 86 ++++++++++--------- .../history/EventLogFileWritersSuite.scala | 36 ++++---- .../deploy/history/EventLogTestHelper.scala | 10 +++ docs/configuration.md | 4 +- 8 files changed, 85 insertions(+), 76 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala index afc520e54d07..6d946dfc2634 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala @@ -256,5 +256,5 @@ class RollingEventLogFilesFileReader( files.filter(isEventLogFile).sortBy { status => getIndex(status.getPath.getName) } } - private def lastEventLogFile: FileStatus = eventLogFiles.reverse.head + private def lastEventLogFile: FileStatus = eventLogFiles.last } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala index 9870b3f83834..1f5b5150bd14 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -88,8 +88,8 @@ abstract class EventLogFileWriter( val isDefaultLocal = defaultFs == null || defaultFs == "file" val uri = path.toUri - /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). - * Therefore, for local files, use FileOutputStream instead. */ + // The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). + // Therefore, for local files, use FileOutputStream instead. val dstream = if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { new FileOutputStream(uri.getPath) @@ -299,7 +299,7 @@ class RollingEventLogFilesWriter( private var countingOutputStream: Option[CountingOutputStream] = None - // seq and event log path will be updated soon in rollEventLogFile, which `start` will call + // index and event log path will be updated soon in rollEventLogFile, which `start` will call private var index: Long = 0L private var currentEventLogFilePath: Path = _ @@ -364,9 +364,9 @@ class RollingEventLogFilesWriter( } object RollingEventLogFilesWriter { - private val EVENT_LOG_DIR_NAME_PREFIX = "eventlog_v2_" - private val EVENT_LOG_FILE_NAME_PREFIX = "events_" - private val APPSTATUS_FILE_NAME_PREFIX = "appstatus_" + private[history] val EVENT_LOG_DIR_NAME_PREFIX = "eventlog_v2_" + private[history] val EVENT_LOG_FILE_NAME_PREFIX = "events_" + private[history] val APPSTATUS_FILE_NAME_PREFIX = "appstatus_" def getAppEventLogDirPath(logBaseDir: URI, appId: String, appAttemptId: Option[String]): Path = new Path(new Path(logBaseDir), EVENT_LOG_DIR_NAME_PREFIX + @@ -408,8 +408,8 @@ object RollingEventLogFilesWriter { } def getIndex(eventLogFileName: String): Long = { - require(eventLogFileName.startsWith(EVENT_LOG_FILE_NAME_PREFIX), "Not a event log file!") - val seq = eventLogFileName.stripPrefix(EVENT_LOG_FILE_NAME_PREFIX).split("_")(0) - seq.toLong + require(eventLogFileName.startsWith(EVENT_LOG_FILE_NAME_PREFIX), "Not an event log file!") + val index = eventLogFileName.stripPrefix(EVENT_LOG_FILE_NAME_PREFIX).split("_")(0) + index.toLong } } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 39ffe5229a46..be218b82789a 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -181,7 +181,8 @@ package object config { private[spark] val EVENT_LOG_ENABLE_ROLLING = ConfigBuilder("spark.eventLog.logRolling.enabled") - .doc("Whether rolling over event log files is enabled.") + .doc("Whether rolling over event log files is enabled. If set to true, it cuts down " + + "each event log file to the configured size.") .booleanConf .createWithDefault(false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 7937f271bd77..683cf3e918ee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -42,7 +42,7 @@ import org.apache.spark.util.{JsonProtocol, Utils} * spark.eventLog.logBlockUpdates.enabled - Whether to log block updates * spark.eventLog.logStageExecutorMetrics.enabled - Whether to log stage executor metrics * - * Event log file writer maintains its own parameters: refer the javadoc of [[EventLogFileWriter]] + * Event log file writer maintains its own parameters: refer the doc of [[EventLogFileWriter]] * and its descendant for more details. */ private[spark] class EventLoggingListener( diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala index bae22dec1e0c..8c79869886dc 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala @@ -22,8 +22,6 @@ import java.net.URI import java.nio.charset.StandardCharsets import java.util.zip.{ZipInputStream, ZipOutputStream} -import scala.collection.mutable - import com.google.common.io.{ByteStreams, Files} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -32,6 +30,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.{LocalSparkContext, SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.EventLogTestHelper._ +import org.apache.spark.deploy.history.RollingEventLogFilesWriter._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.io.CompressionCodec @@ -41,8 +40,7 @@ import org.apache.spark.util.Utils abstract class EventLogFileReadersSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfter with Logging { - protected val fileSystem = Utils.getHadoopFileSystem("/", - SparkHadoopUtil.get.conf) + protected val fileSystem = Utils.getHadoopFileSystem("/", SparkHadoopUtil.get.conf) protected var testDir: File = _ protected var testDirPath: Path = _ @@ -69,7 +67,7 @@ abstract class EventLogFileReadersSuite extends SparkFunSuite with LocalSparkCon } } - def testForPathWithoutSeq( + def testCreateEventLogReaderWithPath( path: Path, isFile: Boolean, expectedClazz: Option[Class[_ <: EventLogFileReader]]): Unit = { @@ -85,7 +83,7 @@ abstract class EventLogFileReadersSuite extends SparkFunSuite with LocalSparkCon assertInstanceOfEventLogReader(expectedClazz, reader) val reader2 = EventLogFileReader(fileSystem, fileSystem.getFileStatus(path)) - assertInstanceOfEventLogReader(expectedClazz, reader) + assertInstanceOfEventLogReader(expectedClazz, reader2) } // path with no last index - single event log @@ -95,25 +93,26 @@ abstract class EventLogFileReadersSuite extends SparkFunSuite with LocalSparkCon // path with last index - rolling event log val reader2 = EventLogFileReader(fileSystem, - new Path(testDirPath, "eventlog_v2_aaa"), Some(3)) + new Path(testDirPath, s"${EVENT_LOG_DIR_NAME_PREFIX}aaa"), Some(3)) assertInstanceOfEventLogReader(Some(classOf[RollingEventLogFilesFileReader]), Some(reader2)) // path - file (both path and FileStatus) val eventLogFile = new Path(testDirPath, "bbb") - testForPathWithoutSeq(eventLogFile, isFile = true, Some(classOf[SingleFileEventLogFileReader])) + testCreateEventLogReaderWithPath(eventLogFile, isFile = true, + Some(classOf[SingleFileEventLogFileReader])) // path - file starting with "." val invalidEventLogFile = new Path(testDirPath, ".bbb") - testForPathWithoutSeq(invalidEventLogFile, isFile = true, None) + testCreateEventLogReaderWithPath(invalidEventLogFile, isFile = true, None) // path - directory with "eventlog_v2_" prefix - val eventLogDir = new Path(testDirPath, "eventlog_v2_ccc") - testForPathWithoutSeq(eventLogDir, isFile = false, + val eventLogDir = new Path(testDirPath, s"${EVENT_LOG_DIR_NAME_PREFIX}ccc") + testCreateEventLogReaderWithPath(eventLogDir, isFile = false, Some(classOf[RollingEventLogFilesFileReader])) // path - directory with no "eventlog_v2_" prefix val invalidEventLogDir = new Path(testDirPath, "ccc") - testForPathWithoutSeq(invalidEventLogDir, isFile = false, None) + testCreateEventLogReaderWithPath(invalidEventLogDir, isFile = false, None) } val allCodecs = Seq(None) ++ @@ -135,8 +134,7 @@ abstract class EventLogFileReadersSuite extends SparkFunSuite with LocalSparkCon dummyData.foreach(writer.writeEvent(_, flushLogger = true)) val logPathIncompleted = getCurrentLogPath(writer.logPath, isCompleted = false) - val readerOpt = EventLogFileReader(fileSystem, - new Path(logPathIncompleted)) + val readerOpt = EventLogFileReader(fileSystem, new Path(logPathIncompleted)) assertAppropriateReader(readerOpt) val reader = readerOpt.get @@ -196,19 +194,19 @@ class SingleFileEventLogFileReaderSuite extends EventLogFileReadersSuite { logPath: Path, compressionCodecShortName: Option[String], isCompleted: Boolean): Unit = { - val stats = fileSystem.getFileStatus(logPath) + val status = fileSystem.getFileStatus(logPath) - assert(stats.isFile) + assert(status.isFile) assert(reader.rootPath === fileSystem.makeQualified(logPath)) assert(reader.lastIndex.isEmpty) - assert(reader.fileSizeForLastIndex === stats.getLen) + assert(reader.fileSizeForLastIndex === status.getLen) assert(reader.completed === isCompleted) - assert(reader.modificationTime === stats.getModificationTime) + assert(reader.modificationTime === status.getModificationTime) assert(reader.listEventLogFiles.length === 1) assert(reader.listEventLogFiles.map(_.getPath.toUri.getPath) === Seq(logPath.toUri.getPath)) assert(reader.compressionCodec === compressionCodecShortName) - assert(reader.totalSize === stats.getLen) + assert(reader.totalSize === status.getLen) val underlyingStream = new ByteArrayOutputStream() Utils.tryWithResource(new ZipOutputStream(underlyingStream)) { os => @@ -245,14 +243,10 @@ class RollingEventLogFilesReaderSuite extends EventLogFileReadersSuite { val dummyString = "dummy" val dummyStringBytesLen = dummyString.getBytes(StandardCharsets.UTF_8).length - val expectedLines = mutable.ArrayBuffer[String]() // write log more than 2k (intended to roll over to 3 files) val repeatCount = Math.floor((1024 * 2) / dummyStringBytesLen).toInt - (0 until repeatCount).foreach { _ => - expectedLines.append(dummyString) - writer.writeEvent(dummyString, flushLogger = true) - } + writeTestEvents(writer, dummyString, repeatCount) val logPathIncompleted = getCurrentLogPath(writer.logPath, isCompleted = false) val readerOpt = EventLogFileReader(fileSystem, @@ -265,7 +259,7 @@ class RollingEventLogFilesReaderSuite extends EventLogFileReadersSuite { val logPathCompleted = getCurrentLogPath(writer.logPath, isCompleted = true) val readerOpt2 = EventLogFileReader(fileSystem, new Path(logPathCompleted)) verifyReader(readerOpt2.get, new Path(logPathCompleted), codecShortName, isCompleted = true) - assert(readerOpt.get.listEventLogFiles.length === 3) + assert(readerOpt2.get.listEventLogFiles.length === 3) } } @@ -293,11 +287,11 @@ class RollingEventLogFilesReaderSuite extends EventLogFileReadersSuite { isCompleted: Boolean): Unit = { import RollingEventLogFilesWriter._ - val stats = fileSystem.getFileStatus(logPath) - assert(stats.isDirectory) + val status = fileSystem.getFileStatus(logPath) + assert(status.isDirectory) - val statsInDir = fileSystem.listStatus(logPath) - val eventFiles = statsInDir.filter(isEventLogFile).sortBy(s => getIndex(s.getPath.getName)) + val statusInDir = fileSystem.listStatus(logPath) + val eventFiles = statusInDir.filter(isEventLogFile).sortBy { s => getIndex(s.getPath.getName) } assert(eventFiles.nonEmpty) val lastEventFile = eventFiles.last val allLen = eventFiles.map(_.getLen).sum @@ -320,25 +314,35 @@ class RollingEventLogFilesReaderSuite extends EventLogFileReadersSuite { Utils.tryWithResource(new ZipInputStream( new ByteArrayInputStream(underlyingStream.toByteArray))) { is => - var entry = is.getNextEntry + val entry = is.getNextEntry assert(entry != null) // directory assert(entry.getName === logPath.getName + "/") - fileSystem.listStatus(logPath).foreach { file => - entry = is.getNextEntry - assert(entry != null) - - assert(entry.getName === logPath.getName + "/" + file.getPath.getName) - - val actual = new String(ByteStreams.toByteArray(is), StandardCharsets.UTF_8) - val expected = Files.toString(new File(logPath.toString, file.getPath.getName), - StandardCharsets.UTF_8) - assert(actual === expected) + val allFileNames = fileSystem.listStatus(logPath).map(_.getPath.getName).toSet + + var count = 0 + var noMoreEntry = false + while (!noMoreEntry) { + val entry = is.getNextEntry + if (entry == null) { + noMoreEntry = true + } else { + count += 1 + + assert(entry.getName.startsWith(logPath.getName + "/")) + val fileName = entry.getName.stripPrefix(logPath.getName + "/") + assert(allFileNames.contains(fileName)) + + val actual = new String(ByteStreams.toByteArray(is), StandardCharsets.UTF_8) + val expected = Files.toString(new File(logPath.toString, fileName), + StandardCharsets.UTF_8) + assert(actual === expected) + } } - assert(is.getNextEntry === null) + assert(count === allFileNames.size) } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala index dbc961b46719..a0ffdb1786e8 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala @@ -46,7 +46,6 @@ abstract class EventLogFileWritersSuite extends SparkFunSuite with LocalSparkCon before { testDir = Utils.createTempDir(namePrefix = s"event log") - testDir.deleteOnExit() testDirPath = new Path(testDir.getAbsolutePath()) } @@ -60,8 +59,7 @@ abstract class EventLogFileWritersSuite extends SparkFunSuite with LocalSparkCon getUniqueApplicationId, None, testDirPath.toUri, conf, SparkHadoopUtil.get.newConfiguration(conf)) val writerClazz = writer.getClass - assert(expectedClazz === writerClazz, - s"default file writer should be $expectedClazz, but $writerClazz") + assert(expectedClazz === writerClazz) } val conf = new SparkConf @@ -231,27 +229,27 @@ class RollingEventLogFilesWriterSuite extends EventLogFileWritersSuite { // happy case with app ID val logDir = RollingEventLogFilesWriter.getAppEventLogDirPath(baseDirUri, appId, None) - assert(s"${baseDirUri.toString}/eventlog_v2_app1" === logDir.toString) + assert(s"${baseDirUri.toString}/${EVENT_LOG_DIR_NAME_PREFIX}${appId}" === logDir.toString) // appstatus: inprogress or completed - assert(s"$logDir/appstatus_app1.inprogress" === + assert(s"$logDir/${APPSTATUS_FILE_NAME_PREFIX}${appId}${EventLogFileWriter.IN_PROGRESS}" === RollingEventLogFilesWriter.getAppStatusFilePath(logDir, appId, appAttemptId, inProgress = true).toString) - assert(s"$logDir/appstatus_app1" === + assert(s"$logDir/${APPSTATUS_FILE_NAME_PREFIX}${appId}" === RollingEventLogFilesWriter.getAppStatusFilePath(logDir, appId, appAttemptId, inProgress = false).toString) // without compression - assert(s"$logDir/events_1_app1" === + assert(s"$logDir/${EVENT_LOG_FILE_NAME_PREFIX}1_${appId}" === RollingEventLogFilesWriter.getEventLogFilePath(logDir, appId, appAttemptId, 1, None).toString) // with compression - assert(s"$logDir/events_1_app1.lzf" === + assert(s"$logDir/${EVENT_LOG_FILE_NAME_PREFIX}1_${appId}.lzf" === RollingEventLogFilesWriter.getEventLogFilePath(logDir, appId, appAttemptId, 1, Some("lzf")).toString) // illegal characters in app ID - assert(s"${baseDirUri.toString}/eventlog_v2_a-fine-mind_dollar_bills__1" === + assert(s"${baseDirUri.toString}/${EVENT_LOG_DIR_NAME_PREFIX}a-fine-mind_dollar_bills__1" === RollingEventLogFilesWriter.getAppEventLogDirPath(baseDirUri, "a fine:mind$dollar{bills}.1", None).toString) } @@ -288,12 +286,12 @@ class RollingEventLogFilesWriterSuite extends EventLogFileWritersSuite { allCodecs.foreach { codecShortName => test(s"rolling event log files - codec $codecShortName") { - def assertEventLogFilesSequence( + def assertEventLogFilesIndex( eventLogFiles: Seq[FileStatus], - expectedLastSequence: Int, + expectedLastIndex: Int, expectedMaxSizeBytes: Long): Unit = { - assert(eventLogFiles.forall(f => f.getLen < expectedMaxSizeBytes)) - assert((1 to expectedLastSequence) === + assert(eventLogFiles.forall(f => f.getLen <= expectedMaxSizeBytes)) + assert((1 to expectedLastIndex) === eventLogFiles.map(f => getIndex(f.getPath.getName))) } @@ -311,24 +309,20 @@ class RollingEventLogFilesWriterSuite extends EventLogFileWritersSuite { val dummyString = "dummy" val dummyStringBytesLen = dummyString.getBytes(StandardCharsets.UTF_8).length - val expectedLines = mutable.ArrayBuffer[String]() // write log more than 2k (intended to roll over to 3 files) val repeatCount = Math.floor((1024 * 2) / dummyStringBytesLen).toInt - (0 until repeatCount).foreach { _ => - expectedLines.append(dummyString) - writer.writeEvent(dummyString, flushLogger = true) - } + val expectedLines = writeTestEvents(writer, dummyString, repeatCount) val logDirPath = getAppEventLogDirPath(testDirPath.toUri, appId, attemptId) val eventLogFiles = listEventLogFiles(logDirPath) - assertEventLogFilesSequence(eventLogFiles, 3, 1024 * 1024) + assertEventLogFilesIndex(eventLogFiles, 3, 1024 * 1024) writer.stop() val eventLogFiles2 = listEventLogFiles(logDirPath) - assertEventLogFilesSequence(eventLogFiles2, 3, 1024 * 1024) + assertEventLogFilesIndex(eventLogFiles2, 3, 1024 * 1024) verifyWriteEventLogFile(appId, attemptId, testDirPath.toUri, codecShortName, expectedLines) @@ -368,6 +362,6 @@ class RollingEventLogFilesWriterSuite extends EventLogFileWritersSuite { private def listEventLogFiles(logDirPath: Path): Seq[FileStatus] = { fileSystem.listStatus(logDirPath).filter(isEventLogFile) - .sortBy(fs => getIndex(fs.getPath.getName)) + .sortBy { fs => getIndex(fs.getPath.getName) } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala index 5608158edf1e..e719e3a150e4 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala @@ -42,4 +42,14 @@ object EventLogTestHelper { conf.set(EVENT_LOG_STAGE_EXECUTOR_METRICS, true) conf } + + def writeTestEvents( + writer: EventLogFileWriter, + eventStr: String, + eventCount: Int): Seq[String] = { + (0 until eventCount).map { _ => + writer.writeEvent(eventStr, flushLogger = true) + eventStr + } + } } diff --git a/docs/configuration.md b/docs/configuration.md index 3420007082bf..5e17f8e03e43 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1013,14 +1013,14 @@ Apart from these, the following properties are also available, and may be useful false Whether rolling over event log files is enabled. If set to true, it cuts down each event - log file to be configured size. + log file to the configured size. spark.eventLog.logRolling.maxFileSize 128m - The max size of event log file to be rolled over. + The max size of event log file before it's rolled over. From f00654257bcb83a95ad6f9178b1906d58e670abb Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sat, 5 Oct 2019 10:54:18 +0900 Subject: [PATCH 11/15] Add new test to check compatibility with LogInfo from v2.4 --- .../history/FsHistoryProviderSuite.scala | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 34ccaaf49a98..c355bcb62771 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -47,6 +47,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.security.GroupMappingServiceProvider import org.apache.spark.status.AppStatusStore +import org.apache.spark.status.KVUtils.KVStoreScalaSerializer import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo} import org.apache.spark.util.{Clock, JsonProtocol, ManualClock, Utils} import org.apache.spark.util.logging.DriverLogger @@ -1252,6 +1253,36 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { } } + test("read old format of LogInfo in Spark 2.4") { + case class LogInfoV24( + logPath: String, + lastProcessed: Long, + appId: Option[String], + attemptId: Option[String], + fileSize: Long) + + val oldObj = LogInfoV24("dummy", System.currentTimeMillis(), Some("hello"), + Some("attempt1"), 100) + + val serializer = new KVStoreScalaSerializer() + val serializedOldObj = serializer.serialize(oldObj) + val deserializedOldObj = serializer.deserialize(serializedOldObj, classOf[LogInfo]) + assert(deserializedOldObj.logPath === oldObj.logPath) + assert(deserializedOldObj.lastProcessed === oldObj.lastProcessed) + assert(deserializedOldObj.appId === oldObj.appId) + assert(deserializedOldObj.attemptId === oldObj.attemptId) + assert(deserializedOldObj.fileSize === oldObj.fileSize) + + // SPARK-25118: added logType: LogType.Value - expected 'null' on old format + assert(deserializedOldObj.logType === null) + + // SPARK-28869: added lastIndex: Option[Long], isComplete: Boolean - expected 'None' and + // 'false' on old format. The default value for isComplete is wrong value for completed app, + // but the value will be corrected once checkForLogs is called. + assert(deserializedOldObj.lastIndex === None) + assert(deserializedOldObj.isComplete === false) + } + /** * Asks the provider to check for logs and calls a function to perform checks on the updated * app list. Example: From 9558b4d90941c85e9ba518733dd723dba2dafabd Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 9 Oct 2019 07:39:51 +0900 Subject: [PATCH 12/15] Reflect review comments --- .../deploy/history/EventLogFileReaders.scala | 30 +++++++++++-------- .../deploy/history/FsHistoryProvider.scala | 30 ++++++++++++------- .../spark/scheduler/ReplayListenerBus.scala | 11 +++++-- .../history/EventLogFileReadersSuite.scala | 7 +---- .../history/EventLogFileWritersSuite.scala | 6 +--- .../deploy/history/EventLogTestHelper.scala | 8 +++-- .../history/FsHistoryProviderSuite.scala | 2 +- 7 files changed, 53 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala index 6d946dfc2634..c8956ed3d423 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala @@ -68,12 +68,15 @@ abstract class EventLogFileReader( def completed: Boolean /** - * Returns the size of file for the last index of event log files, only when - * underlying input stream is DFSInputStream. Otherwise returns None. + * Returns the size of file for the last index (itself for single event log file) of event log + * files, only when underlying input stream is DFSInputStream. Otherwise returns None. */ def fileSizeForLastIndexForDFS: Option[Long] - /** Returns the modification time for the last index of event log files. */ + /** + * Returns the modification time for the last index (itself for single event log file) + * of event log files. + */ def modificationTime: Long /** @@ -149,7 +152,7 @@ object EventLogFileReader { } private def isRollingEventLogs(status: FileStatus): Boolean = { - status.isDirectory && RollingEventLogFilesWriter.isEventLogDir(status) + RollingEventLogFilesWriter.isEventLogDir(status) } } @@ -213,13 +216,18 @@ class RollingEventLogFilesFileReader( private lazy val appStatusFile = files.find(isAppStatusFile).get - override def lastIndex: Option[Long] = { - val maxSeq = files.filter(isEventLogFile) - .map { status => getIndex(status.getPath.getName) } - .max - Some(maxSeq) + private lazy val eventLogFiles: Seq[FileStatus] = { + val eventLogFiles = files.filter(isEventLogFile).sortBy { status => + getIndex(status.getPath.getName) + } + val indices = eventLogFiles.map { file => getIndex(file.getPath.getName) }.sorted + require((indices.head to indices.last) == indices, "Found missing event log file, expected" + + s" indices: ${(indices.head to indices.last)}, actual: ${indices}") + eventLogFiles } + override def lastIndex: Option[Long] = Some(getIndex(lastEventLogFile.getPath.getName)) + override def fileSizeForLastIndex: Long = lastEventLogFile.getLen override def completed: Boolean = { @@ -252,9 +260,5 @@ class RollingEventLogFilesFileReader( override def totalSize: Long = eventLogFiles.map(_.getLen).sum - private def eventLogFiles: Seq[FileStatus] = { - files.filter(isEventLogFile).sortBy { status => getIndex(status.getPath.getName) } - } - private def lastEventLogFile: FileStatus = eventLogFiles.last } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 06887f6c89fb..0b7e695efa87 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -694,11 +694,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) logInfo(s"Parsing $logPath for listing data...") val logFiles = reader.listEventLogFiles - logFiles.foreach { file => - Utils.tryWithResource(EventLogFileReader.openEventLog(file.getPath, fs)) { in => - bus.replay(in, file.getPath.toString, !appCompleted, eventsFilter) - } - } + parseAppEventLogs(logFiles, bus, !appCompleted, eventsFilter) // If enabled above, the listing listener will halt parsing when there's enough information to // create a listing entry. When the app is completed, or fast parsing is disabled, we still need @@ -969,12 +965,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) try { logInfo(s"Parsing ${reader.rootPath} to re-build UI...") - val logFiles = reader.listEventLogFiles - logFiles.foreach { file => - Utils.tryWithResource(EventLogFileReader.openEventLog(file.getPath, fs)) { in => - replayBus.replay(in, file.getPath.toString, maybeTruncated = !reader.completed) - } - } + parseAppEventLogs(reader.listEventLogFiles, replayBus, !reader.completed) trackingStore.close(false) logInfo(s"Finished parsing ${reader.rootPath}") } catch { @@ -986,6 +977,23 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } + private def parseAppEventLogs( + logFiles: Seq[FileStatus], + replayBus: ReplayListenerBus, + maybeTruncated: Boolean, + eventsFilter: ReplayEventsFilter = SELECT_ALL_FILTER): Unit = { + // stop replaying next log files if ReplayListenerBus indicates some error or halt + var continueReplay = true + logFiles.foreach { file => + if (continueReplay) { + Utils.tryWithResource(EventLogFileReader.openEventLog(file.getPath, fs)) { in => + continueReplay = replayBus.replay(in, file.getPath.toString, + maybeTruncated = maybeTruncated, eventsFilter = eventsFilter) + } + } + } + } + /** * Checks whether HDFS is in safe mode. * diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 699042dd967b..60b6fe7a6091 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -48,12 +48,14 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { * @param eventsFilter Filter function to select JSON event strings in the log data stream that * should be parsed and replayed. When not specified, all event strings in the log data * are parsed and replayed. + * @return whether it succeeds to replay the log file entirely without error including + * HaltReplayException. false otherwise. */ def replay( logData: InputStream, sourceName: String, maybeTruncated: Boolean = false, - eventsFilter: ReplayEventsFilter = SELECT_ALL_FILTER): Unit = { + eventsFilter: ReplayEventsFilter = SELECT_ALL_FILTER): Boolean = { val lines = Source.fromInputStream(logData)(Codec.UTF8).getLines() replay(lines, sourceName, maybeTruncated, eventsFilter) } @@ -66,7 +68,7 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { lines: Iterator[String], sourceName: String, maybeTruncated: Boolean, - eventsFilter: ReplayEventsFilter): Unit = { + eventsFilter: ReplayEventsFilter): Boolean = { var currentLine: String = null var lineNumber: Int = 0 val unrecognizedEvents = new scala.collection.mutable.HashSet[String] @@ -114,15 +116,18 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { } } } + true } catch { case e: HaltReplayException => // Just stop replay. - case _: EOFException if maybeTruncated => + false + case _: EOFException if maybeTruncated => false case ioe: IOException => throw ioe case e: Exception => logError(s"Exception parsing Spark event log: $sourceName", e) logError(s"Malformed line #$lineNumber: $currentLine\n") + false } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala index 8c79869886dc..dc87d20ea0a4 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala @@ -241,12 +241,7 @@ class RollingEventLogFilesReaderSuite extends EventLogFileReadersSuite { writer.start() - val dummyString = "dummy" - val dummyStringBytesLen = dummyString.getBytes(StandardCharsets.UTF_8).length - - // write log more than 2k (intended to roll over to 3 files) - val repeatCount = Math.floor((1024 * 2) / dummyStringBytesLen).toInt - writeTestEvents(writer, dummyString, repeatCount) + writeTestEvents(writer, "dummy", 1024 * 2) val logPathIncompleted = getCurrentLogPath(writer.logPath, isCompleted = false) val readerOpt = EventLogFileReader(fileSystem, diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala index a0ffdb1786e8..b90d40427875 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala @@ -307,12 +307,8 @@ class RollingEventLogFilesWriterSuite extends EventLogFileWritersSuite { writer.start() - val dummyString = "dummy" - val dummyStringBytesLen = dummyString.getBytes(StandardCharsets.UTF_8).length - // write log more than 2k (intended to roll over to 3 files) - val repeatCount = Math.floor((1024 * 2) / dummyStringBytesLen).toInt - val expectedLines = writeTestEvents(writer, dummyString, repeatCount) + val expectedLines = writeTestEvents(writer, "dummy", 1024 * 2) val logDirPath = getAppEventLogDirPath(testDirPath.toUri, appId, attemptId) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala index e719e3a150e4..55eddce3968c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala @@ -17,6 +17,8 @@ package org.apache.spark.deploy.history +import java.nio.charset.StandardCharsets + import org.apache.hadoop.fs.Path import org.apache.spark.SparkConf @@ -46,8 +48,10 @@ object EventLogTestHelper { def writeTestEvents( writer: EventLogFileWriter, eventStr: String, - eventCount: Int): Seq[String] = { - (0 until eventCount).map { _ => + desiredSize: Long): Seq[String] = { + val stringLen = eventStr.getBytes(StandardCharsets.UTF_8).length + val repeatCount = Math.floor(desiredSize / stringLen).toInt + (0 until repeatCount).map { _ => writer.writeEvent(eventStr, flushLogger = true) eventStr } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index c355bcb62771..281e6935de37 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -1253,7 +1253,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { } } - test("read old format of LogInfo in Spark 2.4") { + test("backwards compatibility with LogInfo from Spark 2.4") { case class LogInfoV24( logPath: String, lastProcessed: Long, From 540052debcb5e2c0dab4311d2ba8025e76d3b044 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 15 Oct 2019 06:08:52 +0900 Subject: [PATCH 13/15] Reflect review comment --- .../org/apache/spark/deploy/history/EventLogFileWriters.scala | 2 +- .../main/scala/org/apache/spark/internal/config/package.scala | 2 +- .../apache/spark/deploy/history/EventLogFileReadersSuite.scala | 2 +- .../apache/spark/deploy/history/EventLogFileWritersSuite.scala | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala index 1f5b5150bd14..f005f661c75b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -293,7 +293,7 @@ class RollingEventLogFilesWriter( import RollingEventLogFilesWriter._ - private val eventFileMaxLength = sparkConf.get(EVENT_LOG_ROLLED_EVENT_LOG_MAX_FILE_SIZE) + private val eventFileMaxLength = sparkConf.get(EVENT_LOG_ROLLED_LOG_MAX_FILE_SIZE) private val logDirForAppPath = getAppEventLogDirPath(logBaseDir, appId, appAttemptId) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index be218b82789a..5b4680840267 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -186,7 +186,7 @@ package object config { .booleanConf .createWithDefault(false) - private[spark] val EVENT_LOG_ROLLED_EVENT_LOG_MAX_FILE_SIZE = + private[spark] val EVENT_LOG_ROLLED_LOG_MAX_FILE_SIZE = ConfigBuilder("spark.eventLog.logRolling.maxFileSize") .doc("The max size of event log file to be rolled over.") .bytesConf(ByteUnit.BYTE) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala index dc87d20ea0a4..c50f417acc1c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala @@ -234,7 +234,7 @@ class RollingEventLogFilesReaderSuite extends EventLogFileReadersSuite { val conf = getLoggingConf(testDirPath, codecShortName) conf.set(EVENT_LOG_ENABLE_ROLLING, true) - conf.set(EVENT_LOG_ROLLED_EVENT_LOG_MAX_FILE_SIZE.key, "1k") + conf.set(EVENT_LOG_ROLLED_LOG_MAX_FILE_SIZE.key, "1k") val writer = createWriter(appId, attemptId, testDirPath.toUri, conf, SparkHadoopUtil.get.newConfiguration(conf)) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala index b90d40427875..404f50337b11 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala @@ -300,7 +300,7 @@ class RollingEventLogFilesWriterSuite extends EventLogFileWritersSuite { val conf = getLoggingConf(testDirPath, codecShortName) conf.set(EVENT_LOG_ENABLE_ROLLING, true) - conf.set(EVENT_LOG_ROLLED_EVENT_LOG_MAX_FILE_SIZE.key, "1k") + conf.set(EVENT_LOG_ROLLED_LOG_MAX_FILE_SIZE.key, "1k") val writer = createWriter(appId, attemptId, testDirPath.toUri, conf, SparkHadoopUtil.get.newConfiguration(conf)) From 2ff349bcba539c5d1e0ea40ac52fd4b4bf75c3b8 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 16 Oct 2019 11:23:49 +0900 Subject: [PATCH 14/15] Reflect review comments --- .../deploy/history/EventLogFileWriters.scala | 2 +- .../spark/internal/config/package.scala | 8 +++--- .../history/EventLogFileReadersSuite.scala | 6 +++-- .../history/EventLogFileWritersSuite.scala | 27 ++++++++++++++----- docs/configuration.md | 4 +-- 5 files changed, 33 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala index f005f661c75b..3fa5ef94892a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -293,7 +293,7 @@ class RollingEventLogFilesWriter( import RollingEventLogFilesWriter._ - private val eventFileMaxLength = sparkConf.get(EVENT_LOG_ROLLED_LOG_MAX_FILE_SIZE) + private val eventFileMaxLength = sparkConf.get(EVENT_LOG_ROLLING_MAX_FILE_SIZE) private val logDirForAppPath = getAppEventLogDirPath(logBaseDir, appId, appAttemptId) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 5b4680840267..fdb1c8839329 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -180,16 +180,18 @@ package object config { ConfigBuilder("spark.eventLog.longForm.enabled").booleanConf.createWithDefault(false) private[spark] val EVENT_LOG_ENABLE_ROLLING = - ConfigBuilder("spark.eventLog.logRolling.enabled") + ConfigBuilder("spark.eventLog.rolling.enabled") .doc("Whether rolling over event log files is enabled. If set to true, it cuts down " + "each event log file to the configured size.") .booleanConf .createWithDefault(false) - private[spark] val EVENT_LOG_ROLLED_LOG_MAX_FILE_SIZE = - ConfigBuilder("spark.eventLog.logRolling.maxFileSize") + private[spark] val EVENT_LOG_ROLLING_MAX_FILE_SIZE = + ConfigBuilder("spark.eventLog.rolling.maxFileSize") .doc("The max size of event log file to be rolled over.") .bytesConf(ByteUnit.BYTE) + .checkValue(_ >= (1024 * 1024 * 10), "Max file size of event log should be configured to" + + " be at least 10 MiB.") .createWithDefaultString("128m") private[spark] val EXECUTOR_ID = diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala index c50f417acc1c..a2ce4acdaaf3 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala @@ -234,14 +234,16 @@ class RollingEventLogFilesReaderSuite extends EventLogFileReadersSuite { val conf = getLoggingConf(testDirPath, codecShortName) conf.set(EVENT_LOG_ENABLE_ROLLING, true) - conf.set(EVENT_LOG_ROLLED_LOG_MAX_FILE_SIZE.key, "1k") + conf.set(EVENT_LOG_ROLLING_MAX_FILE_SIZE.key, "10m") val writer = createWriter(appId, attemptId, testDirPath.toUri, conf, SparkHadoopUtil.get.newConfiguration(conf)) writer.start() - writeTestEvents(writer, "dummy", 1024 * 2) + // write log more than 20m (intended to roll over to 3 files) + val dummyStr = "dummy" * 1024 + writeTestEvents(writer, dummyStr, 1024 * 1024 * 20) val logPathIncompleted = getCurrentLogPath(writer.logPath, isCompleted = false) val readerOpt = EventLogFileReader(fileSystem, diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala index 404f50337b11..c4b40884eebf 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.deploy.history import java.io.{File, FileOutputStream, IOException} import java.net.URI -import java.nio.charset.StandardCharsets import scala.collection.mutable import scala.io.Source @@ -300,31 +299,47 @@ class RollingEventLogFilesWriterSuite extends EventLogFileWritersSuite { val conf = getLoggingConf(testDirPath, codecShortName) conf.set(EVENT_LOG_ENABLE_ROLLING, true) - conf.set(EVENT_LOG_ROLLED_LOG_MAX_FILE_SIZE.key, "1k") + conf.set(EVENT_LOG_ROLLING_MAX_FILE_SIZE.key, "10m") val writer = createWriter(appId, attemptId, testDirPath.toUri, conf, SparkHadoopUtil.get.newConfiguration(conf)) writer.start() - // write log more than 2k (intended to roll over to 3 files) - val expectedLines = writeTestEvents(writer, "dummy", 1024 * 2) + // write log more than 20m (intended to roll over to 3 files) + val dummyStr = "dummy" * 1024 + val expectedLines = writeTestEvents(writer, dummyStr, 1024 * 1024 * 21) val logDirPath = getAppEventLogDirPath(testDirPath.toUri, appId, attemptId) val eventLogFiles = listEventLogFiles(logDirPath) - assertEventLogFilesIndex(eventLogFiles, 3, 1024 * 1024) + assertEventLogFilesIndex(eventLogFiles, 3, 1024 * 1024 * 10) writer.stop() val eventLogFiles2 = listEventLogFiles(logDirPath) - assertEventLogFilesIndex(eventLogFiles2, 3, 1024 * 1024) + assertEventLogFilesIndex(eventLogFiles2, 3, 1024 * 1024 * 10) verifyWriteEventLogFile(appId, attemptId, testDirPath.toUri, codecShortName, expectedLines) } } + test(s"rolling event log files - the max size of event log file size less than lower limit") { + val appId = getUniqueApplicationId + val attemptId = None + + val conf = getLoggingConf(testDirPath, None) + conf.set(EVENT_LOG_ENABLE_ROLLING, true) + conf.set(EVENT_LOG_ROLLING_MAX_FILE_SIZE.key, "9m") + + val e = intercept[IllegalArgumentException] { + createWriter(appId, attemptId, testDirPath.toUri, conf, + SparkHadoopUtil.get.newConfiguration(conf)) + } + assert(e.getMessage.contains("should be configured to be at least")) + } + override protected def createWriter( appId: String, appAttemptId: Option[String], diff --git a/docs/configuration.md b/docs/configuration.md index 5e17f8e03e43..5fbfbddbd4b8 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1009,7 +1009,7 @@ Apart from these, the following properties are also available, and may be useful - spark.eventLog.logRolling.enabled + spark.eventLog.rolling.enabled false Whether rolling over event log files is enabled. If set to true, it cuts down each event @@ -1017,7 +1017,7 @@ Apart from these, the following properties are also available, and may be useful - spark.eventLog.logRolling.maxFileSize + spark.eventLog.rolling.maxFileSize 128m The max size of event log file before it's rolled over. From a2f631d88504ed360f3f3d3bcb3ceffb83f9c75f Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 17 Oct 2019 05:55:59 +0900 Subject: [PATCH 15/15] Feedback --- .../main/scala/org/apache/spark/internal/config/package.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index fdb1c8839329..b9bccde09bf2 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -190,8 +190,8 @@ package object config { ConfigBuilder("spark.eventLog.rolling.maxFileSize") .doc("The max size of event log file to be rolled over.") .bytesConf(ByteUnit.BYTE) - .checkValue(_ >= (1024 * 1024 * 10), "Max file size of event log should be configured to" + - " be at least 10 MiB.") + .checkValue(_ >= ByteUnit.MiB.toBytes(10), "Max file size of event log should be " + + "configured to be at least 10 MiB.") .createWithDefaultString("128m") private[spark] val EXECUTOR_ID =