From c65d6eadf2e41b9ac621c7af046543ddf61ff2be Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sat, 2 Nov 2019 11:56:51 +0900 Subject: [PATCH 01/28] WIP [SPARK-29779][CORE] Compact old event log files and cleanup - no UTs yet --- ...he.spark.deploy.history.EventFilterBuilder | 1 + .../history/BasicEventFilterBuilder.scala | 146 +++++++++++ .../spark/deploy/history/EventFilter.scala | 82 ++++++ .../history/EventLogFileCompactor.scala | 239 ++++++++++++++++++ .../deploy/history/EventLogFileReaders.scala | 26 +- .../deploy/history/EventLogFileWriters.scala | 13 +- .../deploy/history/FsHistoryProvider.scala | 6 +- .../spark/internal/config/package.scala | 10 + ...he.spark.deploy.history.EventFilterBuilder | 1 + .../history/SQLEventFilterBuilder.scala | 162 ++++++++++++ 10 files changed, 676 insertions(+), 10 deletions(-) create mode 100644 core/src/main/resources/META-INF/services/org.apache.spark.deploy.history.EventFilterBuilder create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala create mode 100644 sql/core/src/main/resources/META-INF/services/org.apache.spark.deploy.history.EventFilterBuilder create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala diff --git a/core/src/main/resources/META-INF/services/org.apache.spark.deploy.history.EventFilterBuilder b/core/src/main/resources/META-INF/services/org.apache.spark.deploy.history.EventFilterBuilder new file mode 100644 index 0000000000000..784e58270ab42 --- /dev/null +++ b/core/src/main/resources/META-INF/services/org.apache.spark.deploy.history.EventFilterBuilder @@ -0,0 +1 @@ +org.apache.spark.deploy.history.BasicEventFilterBuilder \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala new file mode 100644 index 0000000000000..513e749ef499d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala @@ -0,0 +1,146 @@ +/* + * 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 scala.collection.mutable + +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler._ + +class BasicEventFilterBuilder extends SparkListener with EventFilterBuilder { + val liveJobToStages = new mutable.HashMap[Int, Seq[Int]] + val stageToTasks = new mutable.HashMap[Int, mutable.Set[Long]] + + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + liveJobToStages += jobStart.jobId -> jobStart.stageIds + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { + val stages = liveJobToStages.getOrElse(jobEnd.jobId, Seq.empty[Int]) + liveJobToStages -= jobEnd.jobId + stages.foreach { stage => stageToTasks -= stage } + } + + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { + val curTasks = stageToTasks.getOrElseUpdate(taskStart.stageId, + mutable.HashSet[Long]()) + curTasks += taskStart.taskInfo.taskId + } + + override def createFilter(): EventFilter = new BasicEventFilter(this) +} + +class BasicEventFilter(trackListener: BasicEventFilterBuilder) extends EventFilter with Logging { + + private val liveTasks: Set[Long] = trackListener.stageToTasks.values match { + case xs if xs.isEmpty => Set.empty[Long] + case xs => xs.reduce(_ ++ _).toSet + } + + if (log.isDebugEnabled) { + logDebug(s"live jobs : ${trackListener.liveJobToStages.keySet}") + logDebug(s"stages in jobs : ${trackListener.liveJobToStages.values.flatten}") + logDebug(s"stages : ${trackListener.stageToTasks.keySet}") + logDebug(s"tasks in stages : ${trackListener.stageToTasks.values.flatten}") + } + + override def filterStageCompleted(event: SparkListenerStageCompleted): Option[Boolean] = { + Some(trackListener.stageToTasks.contains(event.stageInfo.stageId)) + } + + override def filterStageSubmitted(event: SparkListenerStageSubmitted): Option[Boolean] = { + Some(trackListener.stageToTasks.contains(event.stageInfo.stageId)) + } + + override def filterTaskStart(event: SparkListenerTaskStart): Option[Boolean] = { + Some(liveTasks.contains(event.taskInfo.taskId)) + } + + override def filterTaskGettingResult(event: SparkListenerTaskGettingResult): Option[Boolean] = { + Some(liveTasks.contains(event.taskInfo.taskId)) + } + + override def filterTaskEnd(event: SparkListenerTaskEnd): Option[Boolean] = { + Some(liveTasks.contains(event.taskInfo.taskId)) + } + + override def filterJobStart(event: SparkListenerJobStart): Option[Boolean] = { + Some(trackListener.liveJobToStages.contains(event.jobId)) + } + + override def filterJobEnd(event: SparkListenerJobEnd): Option[Boolean] = { + Some(trackListener.liveJobToStages.contains(event.jobId)) + } + + override def filterUnpersistRDD(event: SparkListenerUnpersistRDD): Option[Boolean] = { + // FIXME: need to track rdd ids? + None + } + + override def filterExecutorMetricsUpdate( + event: SparkListenerExecutorMetricsUpdate): Option[Boolean] = { + // FIXME: need to track live executors? + None + } + + override def filterStageExecutorMetrics( + event: SparkListenerStageExecutorMetrics): Option[Boolean] = { + // FIXME: need to track live executors? + // handle this regardless of liveness of stage - good to restore metrics + None + } + + override def filterExecutorAdded(event: SparkListenerExecutorAdded): Option[Boolean] = { + // FIXME: need to track live executors? + None + } + + override def filterExecutorRemoved(event: SparkListenerExecutorRemoved): Option[Boolean] = { + // FIXME: need to track live executors? + None + } + + override def filterExecutorBlacklisted( + event: SparkListenerExecutorBlacklisted): Option[Boolean] = { + // FIXME: need to track live executors? + None + } + + override def filterExecutorBlacklistedForStage( + event: SparkListenerExecutorBlacklistedForStage): Option[Boolean] = { + // FIXME: need to track live executors? + Some(trackListener.stageToTasks.contains(event.stageId)) + } + + override def filterNodeBlacklistedForStage( + event: SparkListenerNodeBlacklistedForStage): Option[Boolean] = { + // FIXME: need to track live executors? + Some(trackListener.stageToTasks.contains(event.stageId)) + } + + override def filterExecutorUnblacklisted( + event: SparkListenerExecutorUnblacklisted): Option[Boolean] = { + // FIXME: need to track live executors? + None + } + + override def filterSpeculativeTaskSubmitted( + event: SparkListenerSpeculativeTaskSubmitted): Option[Boolean] = { + Some(trackListener.stageToTasks.contains(event.stageId)) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala new file mode 100644 index 0000000000000..da73908bb8ad1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala @@ -0,0 +1,82 @@ +/* + * 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.spark.scheduler._ + +trait EventFilterBuilder extends SparkListenerInterface { + def createFilter(): EventFilter +} + +trait EventFilter { + def filterStageCompleted(event: SparkListenerStageCompleted): Option[Boolean] = None + + def filterStageSubmitted(event: SparkListenerStageSubmitted): Option[Boolean] = None + + def filterTaskStart(event: SparkListenerTaskStart): Option[Boolean] = None + + def filterTaskGettingResult(event: SparkListenerTaskGettingResult): Option[Boolean] = None + + def filterTaskEnd(event: SparkListenerTaskEnd): Option[Boolean] = None + + def filterJobStart(event: SparkListenerJobStart): Option[Boolean] = None + + def filterJobEnd(event: SparkListenerJobEnd): Option[Boolean] = None + + def filterEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Option[Boolean] = None + + def filterBlockManagerAdded(event: SparkListenerBlockManagerAdded): Option[Boolean] = None + + def filterBlockManagerRemoved(event: SparkListenerBlockManagerRemoved): Option[Boolean] = None + + def filterUnpersistRDD(event: SparkListenerUnpersistRDD): Option[Boolean] = None + + def filterApplicationStart(event: SparkListenerApplicationStart): Option[Boolean] = None + + def filterApplicationEnd(event: SparkListenerApplicationEnd): Option[Boolean] = None + + def filterExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Option[Boolean] = None + + def filterStageExecutorMetrics(event: SparkListenerStageExecutorMetrics): Option[Boolean] = None + + def filterExecutorAdded(event: SparkListenerExecutorAdded): Option[Boolean] = None + + def filterExecutorRemoved(event: SparkListenerExecutorRemoved): Option[Boolean] = None + + def filterExecutorBlacklisted(event: SparkListenerExecutorBlacklisted): Option[Boolean] = None + + def filterExecutorBlacklistedForStage( + event: SparkListenerExecutorBlacklistedForStage): Option[Boolean] = None + + def filterNodeBlacklistedForStage( + event: SparkListenerNodeBlacklistedForStage): Option[Boolean] = None + + def filterExecutorUnblacklisted(event: SparkListenerExecutorUnblacklisted): Option[Boolean] = None + + def filterNodeBlacklisted(event: SparkListenerNodeBlacklisted): Option[Boolean] = None + + def filterNodeUnblacklisted(event: SparkListenerNodeUnblacklisted): Option[Boolean] = None + + def filterBlockUpdated(event: SparkListenerBlockUpdated): Option[Boolean] = None + + def filterSpeculativeTaskSubmitted( + event: SparkListenerSpeculativeTaskSubmitted): Option[Boolean] = None + + def filterOtherEvent(event: SparkListenerEvent): Option[Boolean] = None +} + diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala new file mode 100644 index 0000000000000..22871a1fcfd17 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -0,0 +1,239 @@ +/* + * 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.IOException +import java.net.URI +import java.util.ServiceLoader + +import scala.collection.JavaConverters._ +import scala.io.{Codec, Source} + +import com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.json4s.jackson.JsonMethods.parse + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN +import org.apache.spark.scheduler._ +import org.apache.spark.util.{JsonProtocol, Utils} + +class EventLogFileCompactor( + sparkConf: SparkConf, + hadoopConf: Configuration, + fs: FileSystem) extends Logging { + + private val maxFilesToRetain: Int = sparkConf.get(EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN) + + // FIXME: javadoc - caller should provide event log files (either compacted or original) + // sequentially if the last event log file is already a compacted file, everything + // will be skipped + def compact(eventLogFiles: Seq[FileStatus]): Seq[FileStatus] = { + if (eventLogFiles.isEmpty) { + return Seq.empty[FileStatus] + } + + // skip everything if the last file is already a compacted file + if (EventLogFileWriter.isCompacted(eventLogFiles.last.getPath)) { + return Seq(eventLogFiles.last) + } + + val (filesToCompact, filesToRetain) = findFilesToCompact(eventLogFiles) + if (filesToCompact.isEmpty) { + filesToRetain + } else { + // first pass + val bus = new ReplayListenerBus() + + val builders = ServiceLoader.load(classOf[EventFilterBuilder], + Utils.getContextOrSparkClassLoader).asScala + builders.foreach(bus.addListener) + + filesToCompact.foreach { log => + Utils.tryWithResource(EventLogFileReader.openEventLog(log.getPath, fs)) { in => + bus.replay(in, log.getPath.getName) + } + } + + // second pass + val rewriter = new FilteredEventLogFileRewriter(sparkConf, hadoopConf, + filesToCompact, fs, builders.map(_.createFilter()).toSeq) + rewriter.start() + rewriter.rewrite() + rewriter.stop() + + // cleanup files which are replaced with new compacted file. + cleanupCompactedFiles(filesToCompact) + + fs.getFileStatus(new Path(rewriter.logPath)) :: filesToRetain.toList + } + } + + private def cleanupCompactedFiles(files: Seq[FileStatus]): Unit = { + files.foreach { file => + try { + fs.delete(file.getPath, true) + } catch { + case _: IOException => logWarning(s"Failed to remove ${file.getPath} / skip removing.") + } + } + } + + private def findFilesToCompact( + eventLogFiles: Seq[FileStatus]): (Seq[FileStatus], Seq[FileStatus]) = { + val lastCompactedFileIdx = eventLogFiles.lastIndexWhere { fs => + EventLogFileWriter.isCompacted(fs.getPath) + } + + val files = if (lastCompactedFileIdx > -1) { + eventLogFiles.drop(lastCompactedFileIdx) + } else { + eventLogFiles + } + + if (files.length > maxFilesToRetain) { + (files.dropRight(maxFilesToRetain), files.takeRight(maxFilesToRetain)) + } else { + (Seq.empty, files) + } + } +} + +class FilteredEventLogFileRewriter( + sparkConf: SparkConf, + hadoopConf: Configuration, + eventLogFiles: Seq[FileStatus], + fs: FileSystem, + filters: Seq[EventFilter]) extends Logging { + + require(eventLogFiles.nonEmpty) + + private val targetEventLogFilePath = eventLogFiles.last.getPath + private val logWriter: CompactedEventLogFileWriter = new CompactedEventLogFileWriter( + targetEventLogFilePath, "dummy", None, targetEventLogFilePath.getParent.toUri, + sparkConf, hadoopConf) + + def logPath: String = logWriter.logPath + + def start(): Unit = { + logWriter.start() + } + + def stop(): Unit = { + logWriter.stop() + } + + def rewrite(): Unit = { + eventLogFiles.foreach(rewriteFile) + } + + private def rewriteFile(fileStatus: FileStatus): Unit = { + Utils.tryWithResource(EventLogFileReader.openEventLog(fileStatus.getPath, fs)) { in => + val lines = Source.fromInputStream(in)(Codec.UTF8).getLines() + + var currentLine: String = null + var lineNumber: Int = 0 + + try { + val lineEntries = lines.zipWithIndex + + while (lineEntries.hasNext) { + try { + val entry = lineEntries.next() + + currentLine = entry._1 + lineNumber = entry._2 + 1 + + val event = JsonProtocol.sparkEventFromJson(parse(currentLine)) + if (checkFilters(event)) { + logWriter.writeLine(currentLine) + } + } catch { + // ignore any exception occurred from unidentified json + // just skip handling and write the line + case _: ClassNotFoundException => logWriter.writeLine(currentLine) + case _: UnrecognizedPropertyException => logWriter.writeLine(currentLine) + } + } + true + } catch { + case e: Exception => + logError(s"Exception parsing Spark event log: ${fileStatus.getPath.getName}", e) + logError(s"Malformed line #$lineNumber: $currentLine\n") + throw e + } + } + } + + private def checkFilters(event: SparkListenerEvent): Boolean = { + val results = filters.flatMap(filter => applyFilter(filter, event)) + results.isEmpty || results.forall(_ == true) + } + + private def applyFilter(filter: EventFilter, event: SparkListenerEvent): Option[Boolean] = { + event match { + case event: SparkListenerStageSubmitted => filter.filterStageSubmitted(event) + case event: SparkListenerStageCompleted => filter.filterStageCompleted(event) + case event: SparkListenerJobStart => filter.filterJobStart(event) + case event: SparkListenerJobEnd => filter.filterJobEnd(event) + case event: SparkListenerTaskStart => filter.filterTaskStart(event) + case event: SparkListenerTaskGettingResult => filter.filterTaskGettingResult(event) + case event: SparkListenerTaskEnd => filter.filterTaskEnd(event) + case event: SparkListenerEnvironmentUpdate => filter.filterEnvironmentUpdate(event) + case event: SparkListenerBlockManagerAdded => filter.filterBlockManagerAdded(event) + case event: SparkListenerBlockManagerRemoved => filter.filterBlockManagerRemoved(event) + case event: SparkListenerUnpersistRDD => filter.filterUnpersistRDD(event) + case event: SparkListenerApplicationStart => filter.filterApplicationStart(event) + case event: SparkListenerApplicationEnd => filter.filterApplicationEnd(event) + case event: SparkListenerExecutorMetricsUpdate => filter.filterExecutorMetricsUpdate(event) + case event: SparkListenerStageExecutorMetrics => filter.filterStageExecutorMetrics(event) + case event: SparkListenerExecutorAdded => filter.filterExecutorAdded(event) + case event: SparkListenerExecutorRemoved => filter.filterExecutorRemoved(event) + case event: SparkListenerExecutorBlacklistedForStage => + filter.filterExecutorBlacklistedForStage(event) + case event: SparkListenerNodeBlacklistedForStage => + filter.filterNodeBlacklistedForStage(event) + case event: SparkListenerExecutorBlacklisted => filter.filterExecutorBlacklisted(event) + case event: SparkListenerExecutorUnblacklisted => filter.filterExecutorUnblacklisted(event) + case event: SparkListenerNodeBlacklisted => filter.filterNodeBlacklisted(event) + case event: SparkListenerNodeUnblacklisted => filter.filterNodeUnblacklisted(event) + case event: SparkListenerBlockUpdated => filter.filterBlockUpdated(event) + case event: SparkListenerSpeculativeTaskSubmitted => + filter.filterSpeculativeTaskSubmitted(event) + case _ => filter.filterOtherEvent(event) + } + } +} + +class CompactedEventLogFileWriter( + originalFilePath: Path, + appId: String, + appAttemptId: Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration) + extends SingleEventLogFileWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) { + + override val logPath: String = originalFilePath.toUri.toString + EventLogFileWriter.COMPACTED + + override def writeLine(line: String, flushLogger: Boolean): Unit = { + super.writeLine(line, flushLogger) + } +} 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 c8956ed3d423d..88cbeccab8e25 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 @@ -173,7 +173,8 @@ class SingleFileEventLogFileReader( override def fileSizeForLastIndex: Long = status.getLen - override def completed: Boolean = !rootPath.getName.endsWith(EventLogFileWriter.IN_PROGRESS) + override def completed: Boolean = !rootPath.getName.stripSuffix(EventLogFileWriter.COMPACTED) + .endsWith(EventLogFileWriter.IN_PROGRESS) override def fileSizeForLastIndexForDFS: Option[Long] = { if (completed) { @@ -218,12 +219,29 @@ class RollingEventLogFilesFileReader( private lazy val eventLogFiles: Seq[FileStatus] = { val eventLogFiles = files.filter(isEventLogFile).sortBy { status => - getIndex(status.getPath.getName) + val filePath = status.getPath + var idx = getIndex(status.getPath.getName) + 0.0d + // trick to place compacted file later than normal file if index is same. + if (EventLogFileWriter.isCompacted(filePath)) { + idx += 0.1 + } + idx + } + + val lastCompactedFileIdx = eventLogFiles.lastIndexWhere { fs => + EventLogFileWriter.isCompacted(fs.getPath) } - val indices = eventLogFiles.map { file => getIndex(file.getPath.getName) }.sorted + + val filesToRead = if (lastCompactedFileIdx > -1) { + eventLogFiles.drop(lastCompactedFileIdx) + } else { + eventLogFiles + } + + val indices = filesToRead.map { file => getIndex(file.getPath.getName) } require((indices.head to indices.last) == indices, "Found missing event log file, expected" + s" indices: ${(indices.head to indices.last)}, actual: ${indices}") - eventLogFiles + filesToRead } override def lastIndex: Option[Long] = Some(getIndex(lastEventLogFile.getPath.getName)) 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 3fa5ef94892aa..d5f413e681a2c 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 @@ -113,9 +113,9 @@ abstract class EventLogFileWriter( } } - protected def writeJson(json: String, flushLogger: Boolean = false): Unit = { + protected def writeLine(line: String, flushLogger: Boolean = false): Unit = { // scalastyle:off println - writer.foreach(_.println(json)) + writer.foreach(_.println(line)) // scalastyle:on println if (flushLogger) { writer.foreach(_.flush()) @@ -164,6 +164,7 @@ abstract class EventLogFileWriter( object EventLogFileWriter { // Suffix applied to the names of files still being written by applications. val IN_PROGRESS = ".inprogress" + val COMPACTED = ".compact" val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) @@ -192,9 +193,11 @@ object EventLogFileWriter { 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) + val logName = log.getName.stripSuffix(COMPACTED).stripSuffix(IN_PROGRESS) logName.split("\\.").tail.lastOption } + + def isCompacted(log: Path): Boolean = log.getName.endsWith(COMPACTED) } /** @@ -222,7 +225,7 @@ class SingleEventLogFileWriter( } override def writeEvent(eventJson: String, flushLogger: Boolean = false): Unit = { - writeJson(eventJson, flushLogger) + writeLine(eventJson, flushLogger) } /** @@ -327,7 +330,7 @@ class RollingEventLogFilesWriter( } } - writeJson(eventJson, flushLogger) + writeLine(eventJson, flushLogger) } private def rollEventLogFile(): Unit = { 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 e2f3314bc8595..432fc92671806 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 @@ -160,6 +160,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) new HistoryServerDiskManager(conf, path, listing, clock) } + private val fileCompactor = new EventLogFileCompactor(conf, hadoopConf, fs) + private val blacklist = new ConcurrentHashMap[String, Long] // Visible for testing @@ -963,8 +965,10 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } replayBus.addListener(listener) try { + logInfo(s"Compacting ${reader.rootPath}...") + val newEventLogFiles = fileCompactor.compact(reader.listEventLogFiles) logInfo(s"Parsing ${reader.rootPath} to re-build UI...") - parseAppEventLogs(reader.listEventLogFiles, replayBus, !reader.completed) + parseAppEventLogs(newEventLogFiles, replayBus, !reader.completed) trackingStore.close(false) logInfo(s"Finished parsing ${reader.rootPath}") } catch { 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 8e8e36dbda94e..cea7bc1bbdef6 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 @@ -194,6 +194,16 @@ package object config { "configured to be at least 10 MiB.") .createWithDefaultString("128m") + private[spark] val EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN = + ConfigBuilder("spark.eventLog.rolling.maxFilesToRetain") + .doc("The maximum number of event log files which will be retained as non-compacted. " + + "By default, all event log files will be retained. Please set the configuration " + + s"and ${EVENT_LOG_ROLLING_MAX_FILE_SIZE.key} accordingly if you want to control " + + "the overall size of event log files.") + .intConf + .checkValue(_ > 0, "Max event log files to retain should be higher than 0.") + .createWithDefault(Integer.MAX_VALUE) + private[spark] val EXECUTOR_ID = ConfigBuilder("spark.executor.id").stringConf.createOptional diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.deploy.history.EventFilterBuilder b/sql/core/src/main/resources/META-INF/services/org.apache.spark.deploy.history.EventFilterBuilder new file mode 100644 index 0000000000000..5025616b752d1 --- /dev/null +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.deploy.history.EventFilterBuilder @@ -0,0 +1 @@ +org.apache.spark.sql.execution.history.SQLEventFilterBuilder \ No newline at end of file diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala new file mode 100644 index 0000000000000..e54b241bb84d9 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala @@ -0,0 +1,162 @@ +/* + * 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.sql.execution.history + +import scala.collection.mutable + +import org.apache.spark.deploy.history.{EventFilter, EventFilterBuilder} +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler._ +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.ui._ +import org.apache.spark.sql.streaming.StreamingQueryListener + +class SQLEventFilterBuilder extends SparkListener with EventFilterBuilder { + val liveExecutionToJobs = new mutable.HashMap[Long, mutable.Set[Int]] + val jobToStages = new mutable.HashMap[Int, Seq[Int]] + val stageToTasks = new mutable.HashMap[Int, mutable.Set[Long]] + + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + val executionIdString = jobStart.properties.getProperty(SQLExecution.EXECUTION_ID_KEY) + if (executionIdString == null) { + // This is not a job created by SQL + return + } + + val executionId = executionIdString.toLong + val jobId = jobStart.jobId + + val jobsForExecution = liveExecutionToJobs.getOrElseUpdate(executionId, + mutable.HashSet[Int]()) + jobsForExecution += jobId + + jobToStages += jobStart.jobId -> jobStart.stageIds + jobStart.stageIds.foreach { stageId => stageToTasks += stageId -> mutable.HashSet[Long]() } + } + + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { + if (stageToTasks.contains(taskStart.stageId)) { + val curTasks = stageToTasks(taskStart.stageId) + curTasks += taskStart.taskInfo.taskId + } + } + + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { + case e: SparkListenerSQLExecutionStart => onExecutionStart(e) + case e: SparkListenerSQLExecutionEnd => onExecutionEnd(e) + case _ => // Ignore + } + + private def onExecutionStart(event: SparkListenerSQLExecutionStart): Unit = { + liveExecutionToJobs += event.executionId -> mutable.HashSet[Int]() + } + + private def onExecutionEnd(event: SparkListenerSQLExecutionEnd): Unit = { + val jobs = liveExecutionToJobs.getOrElse(event.executionId, mutable.HashSet[Int]()) + liveExecutionToJobs -= event.executionId + + val stages = jobToStages.filter(kv => jobs.contains(kv._1)).values.flatten + jobToStages --= jobs + stageToTasks --= stages + } + + override def createFilter(): EventFilter = { + new SQLLiveEntitiesEventFilter(this) + } +} + +class SQLLiveEntitiesEventFilter(trackListener: SQLEventFilterBuilder) + extends EventFilter with Logging { + + private val liveTasks: Set[Long] = trackListener.stageToTasks.values match { + case xs if xs.isEmpty => Set.empty[Long] + case xs => xs.reduce(_ ++ _).toSet + } + + if (log.isDebugEnabled) { + logDebug(s"live executions : ${trackListener.liveExecutionToJobs.keySet}") + logDebug(s"jobs in live executions : ${trackListener.liveExecutionToJobs.values.flatten}") + logDebug(s"jobs : ${trackListener.jobToStages.keySet}") + logDebug(s"stages in jobs : ${trackListener.jobToStages.values.flatten}") + logDebug(s"stages : ${trackListener.stageToTasks.keySet}") + logDebug(s"tasks in stages : ${trackListener.stageToTasks.values.flatten}") + } + + override def filterStageCompleted(event: SparkListenerStageCompleted): Option[Boolean] = { + Some(trackListener.stageToTasks.contains(event.stageInfo.stageId)) + } + + override def filterStageSubmitted(event: SparkListenerStageSubmitted): Option[Boolean] = { + Some(trackListener.stageToTasks.contains(event.stageInfo.stageId)) + } + + override def filterTaskStart(event: SparkListenerTaskStart): Option[Boolean] = { + Some(liveTasks.contains(event.taskInfo.taskId)) + } + + override def filterTaskGettingResult(event: SparkListenerTaskGettingResult): Option[Boolean] = { + Some(liveTasks.contains(event.taskInfo.taskId)) + } + + override def filterTaskEnd(event: SparkListenerTaskEnd): Option[Boolean] = { + Some(liveTasks.contains(event.taskInfo.taskId)) + } + + override def filterJobStart(event: SparkListenerJobStart): Option[Boolean] = { + Some(trackListener.jobToStages.contains(event.jobId)) + } + + override def filterJobEnd(event: SparkListenerJobEnd): Option[Boolean] = { + Some(trackListener.jobToStages.contains(event.jobId)) + } + + override def filterExecutorMetricsUpdate( + event: SparkListenerExecutorMetricsUpdate): Option[Boolean] = { + Some(event.accumUpdates.exists { case (_, stageId, _, _) => + trackListener.stageToTasks.contains(stageId) + }) + } + + override def filterOtherEvent(event: SparkListenerEvent): Option[Boolean] = event match { + case e: SparkListenerSQLExecutionStart => filterExecutionStart(e) + case e: SparkListenerSQLAdaptiveExecutionUpdate => filterAdaptiveExecutionUpdate(e) + case e: SparkListenerSQLExecutionEnd => filterExecutionEnd(e) + case e: SparkListenerDriverAccumUpdates => filterDriverAccumUpdates(e) + + // these events are for finished batches so safer to ignore + case _: StreamingQueryListener.QueryProgressEvent => Some(false) + case _ => None + } + + def filterExecutionStart(event: SparkListenerSQLExecutionStart): Option[Boolean] = { + Some(trackListener.liveExecutionToJobs.contains(event.executionId)) + } + + def filterAdaptiveExecutionUpdate( + event: SparkListenerSQLAdaptiveExecutionUpdate): Option[Boolean] = { + Some(trackListener.liveExecutionToJobs.contains(event.executionId)) + } + + def filterExecutionEnd(event: SparkListenerSQLExecutionEnd): Option[Boolean] = { + Some(trackListener.liveExecutionToJobs.contains(event.executionId)) + } + + def filterDriverAccumUpdates(event: SparkListenerDriverAccumUpdates): Option[Boolean] = { + Some(trackListener.liveExecutionToJobs.contains(event.executionId)) + } +} From bb485c71ca9d44eb96d3066a938a9988e52fe417 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sun, 10 Nov 2019 14:04:10 +0900 Subject: [PATCH 02/28] Reflect review comments --- .../spark/deploy/history/EventLogFileCompactor.scala | 8 ++------ .../apache/spark/deploy/history/EventLogFileReaders.scala | 7 +------ 2 files changed, 3 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index 22871a1fcfd17..291170941b2ca 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -101,12 +101,7 @@ class EventLogFileCompactor( val lastCompactedFileIdx = eventLogFiles.lastIndexWhere { fs => EventLogFileWriter.isCompacted(fs.getPath) } - - val files = if (lastCompactedFileIdx > -1) { - eventLogFiles.drop(lastCompactedFileIdx) - } else { - eventLogFiles - } + val files = eventLogFiles.drop(lastCompactedFileIdx) if (files.length > maxFilesToRetain) { (files.dropRight(maxFilesToRetain), files.takeRight(maxFilesToRetain)) @@ -233,6 +228,7 @@ class CompactedEventLogFileWriter( override val logPath: String = originalFilePath.toUri.toString + EventLogFileWriter.COMPACTED + // override to make writeLine method be 'public' only for this class override def writeLine(line: String, flushLogger: Boolean): Unit = { super.writeLine(line, flushLogger) } 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 88cbeccab8e25..597fd3f2671f9 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 @@ -231,12 +231,7 @@ class RollingEventLogFilesFileReader( val lastCompactedFileIdx = eventLogFiles.lastIndexWhere { fs => EventLogFileWriter.isCompacted(fs.getPath) } - - val filesToRead = if (lastCompactedFileIdx > -1) { - eventLogFiles.drop(lastCompactedFileIdx) - } else { - eventLogFiles - } + val filesToRead = eventLogFiles.drop(lastCompactedFileIdx) val indices = filesToRead.map { file => getIndex(file.getPath.getName) } require((indices.head to indices.last) == indices, "Found missing event log file, expected" + From 87404de6f6f08978376a8e95a046c9ba7f93f9c8 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 11 Nov 2019 11:07:11 +0900 Subject: [PATCH 03/28] Address FIXMEs --- .../history/BasicEventFilterBuilder.scala | 57 ++++++++----------- 1 file changed, 25 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala index 513e749ef499d..6bfb0e1687310 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala @@ -25,6 +25,8 @@ import org.apache.spark.scheduler._ class BasicEventFilterBuilder extends SparkListener with EventFilterBuilder { val liveJobToStages = new mutable.HashMap[Int, Seq[Int]] val stageToTasks = new mutable.HashMap[Int, mutable.Set[Long]] + val stageToRDDs = new mutable.HashMap[Int, mutable.Set[Int]] + val liveExecutors = new mutable.HashSet[String] override def onJobStart(jobStart: SparkListenerJobStart): Unit = { liveJobToStages += jobStart.jobId -> jobStart.stageIds @@ -33,7 +35,10 @@ class BasicEventFilterBuilder extends SparkListener with EventFilterBuilder { override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { val stages = liveJobToStages.getOrElse(jobEnd.jobId, Seq.empty[Int]) liveJobToStages -= jobEnd.jobId - stages.foreach { stage => stageToTasks -= stage } + stages.foreach { stage => + stageToTasks -= stage + stageToRDDs -= stage + } } override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { @@ -42,6 +47,14 @@ class BasicEventFilterBuilder extends SparkListener with EventFilterBuilder { curTasks += taskStart.taskInfo.taskId } + override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { + liveExecutors += executorAdded.executorId + } + + override def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit = { + liveExecutors -= executorRemoved.executorId + } + override def createFilter(): EventFilter = new BasicEventFilter(this) } @@ -52,6 +65,11 @@ class BasicEventFilter(trackListener: BasicEventFilterBuilder) extends EventFilt case xs => xs.reduce(_ ++ _).toSet } + private val liveRDDs: Set[Int] = trackListener.stageToRDDs.values match { + case xs if xs.isEmpty => Set.empty[Int] + case xs => xs.reduce(_ ++ _).toSet + } + if (log.isDebugEnabled) { logDebug(s"live jobs : ${trackListener.liveJobToStages.keySet}") logDebug(s"stages in jobs : ${trackListener.liveJobToStages.values.flatten}") @@ -88,55 +106,30 @@ class BasicEventFilter(trackListener: BasicEventFilterBuilder) extends EventFilt } override def filterUnpersistRDD(event: SparkListenerUnpersistRDD): Option[Boolean] = { - // FIXME: need to track rdd ids? - None - } - - override def filterExecutorMetricsUpdate( - event: SparkListenerExecutorMetricsUpdate): Option[Boolean] = { - // FIXME: need to track live executors? - None + Some(liveRDDs.contains(event.rddId)) } override def filterStageExecutorMetrics( event: SparkListenerStageExecutorMetrics): Option[Boolean] = { - // FIXME: need to track live executors? - // handle this regardless of liveness of stage - good to restore metrics - None + Some(trackListener.liveExecutors.contains(event.execId)) } override def filterExecutorAdded(event: SparkListenerExecutorAdded): Option[Boolean] = { - // FIXME: need to track live executors? - None + Some(trackListener.liveExecutors.contains(event.executorId)) } override def filterExecutorRemoved(event: SparkListenerExecutorRemoved): Option[Boolean] = { - // FIXME: need to track live executors? - None + Some(trackListener.liveExecutors.contains(event.executorId)) } override def filterExecutorBlacklisted( event: SparkListenerExecutorBlacklisted): Option[Boolean] = { - // FIXME: need to track live executors? - None - } - - override def filterExecutorBlacklistedForStage( - event: SparkListenerExecutorBlacklistedForStage): Option[Boolean] = { - // FIXME: need to track live executors? - Some(trackListener.stageToTasks.contains(event.stageId)) - } - - override def filterNodeBlacklistedForStage( - event: SparkListenerNodeBlacklistedForStage): Option[Boolean] = { - // FIXME: need to track live executors? - Some(trackListener.stageToTasks.contains(event.stageId)) + Some(trackListener.liveExecutors.contains(event.executorId)) } override def filterExecutorUnblacklisted( event: SparkListenerExecutorUnblacklisted): Option[Boolean] = { - // FIXME: need to track live executors? - None + Some(trackListener.liveExecutors.contains(event.executorId)) } override def filterSpeculativeTaskSubmitted( From 2944416efd7c5a16943b079128ca9b711c2d131a Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 11 Nov 2019 17:52:29 +0900 Subject: [PATCH 04/28] Add UTs for BasicEventFilter/BasicEventFilterBuilder --- .../history/BasicEventFilterBuilder.scala | 87 ++++--- .../history/EventLogFileCompactor.scala | 2 + .../BasicEventFilterBuilderSuite.scala | 236 ++++++++++++++++++ .../history/BasicEventFilterSuite.scala | 170 +++++++++++++ .../spark/status/AppStatusListenerSuite.scala | 37 +-- .../status/ListenerEventsTestHelper.scala | 153 ++++++++++++ .../history/SQLEventFilterBuilder.scala | 2 + 7 files changed, 620 insertions(+), 67 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala index 6bfb0e1687310..ba5b192740844 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala @@ -22,67 +22,82 @@ import scala.collection.mutable import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ -class BasicEventFilterBuilder extends SparkListener with EventFilterBuilder { - val liveJobToStages = new mutable.HashMap[Int, Seq[Int]] - val stageToTasks = new mutable.HashMap[Int, mutable.Set[Long]] - val stageToRDDs = new mutable.HashMap[Int, mutable.Set[Int]] - val liveExecutors = new mutable.HashSet[String] +private[spark] class BasicEventFilterBuilder extends SparkListener with EventFilterBuilder { + private val _liveJobToStages = new mutable.HashMap[Int, Seq[Int]] + private val _stageToTasks = new mutable.HashMap[Int, mutable.Set[Long]] + private val _stageToRDDs = new mutable.HashMap[Int, Seq[Int]] + private val _liveExecutors = new mutable.HashSet[String] + + def liveJobToStages: Map[Int, Seq[Int]] = _liveJobToStages.toMap + def stageToTasks: Map[Int, Set[Long]] = _stageToTasks.mapValues(_.toSet).toMap + def stageToRDDs: Map[Int, Seq[Int]] = _stageToRDDs.toMap + def liveExecutors: Set[String] = _liveExecutors.toSet override def onJobStart(jobStart: SparkListenerJobStart): Unit = { - liveJobToStages += jobStart.jobId -> jobStart.stageIds + _liveJobToStages += jobStart.jobId -> jobStart.stageIds } override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { - val stages = liveJobToStages.getOrElse(jobEnd.jobId, Seq.empty[Int]) - liveJobToStages -= jobEnd.jobId + val stages = _liveJobToStages.getOrElse(jobEnd.jobId, Seq.empty[Int]) + _liveJobToStages -= jobEnd.jobId stages.foreach { stage => - stageToTasks -= stage - stageToRDDs -= stage + _stageToTasks -= stage + _stageToRDDs -= stage } } + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { + _stageToRDDs.getOrElseUpdate(stageSubmitted.stageInfo.stageId, + stageSubmitted.stageInfo.rddInfos.map(_.id)) + } + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { - val curTasks = stageToTasks.getOrElseUpdate(taskStart.stageId, + val curTasks = _stageToTasks.getOrElseUpdate(taskStart.stageId, mutable.HashSet[Long]()) curTasks += taskStart.taskInfo.taskId } override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { - liveExecutors += executorAdded.executorId + _liveExecutors += executorAdded.executorId } override def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit = { - liveExecutors -= executorRemoved.executorId + _liveExecutors -= executorRemoved.executorId } - override def createFilter(): EventFilter = new BasicEventFilter(this) + override def createFilter(): EventFilter = BasicEventFilter(this) } -class BasicEventFilter(trackListener: BasicEventFilterBuilder) extends EventFilter with Logging { +private[spark] class BasicEventFilter( + liveJobToStages: Map[Int, Seq[Int]], + stageToTasks: Map[Int, Set[Long]], + stageToRDDs: Map[Int, Seq[Int]], + liveExecutors: Set[String]) extends EventFilter with Logging { - private val liveTasks: Set[Long] = trackListener.stageToTasks.values match { + private val liveTasks: Set[Long] = stageToTasks.values match { case xs if xs.isEmpty => Set.empty[Long] case xs => xs.reduce(_ ++ _).toSet } - private val liveRDDs: Set[Int] = trackListener.stageToRDDs.values match { + private val liveRDDs: Set[Int] = stageToRDDs.values match { case xs if xs.isEmpty => Set.empty[Int] case xs => xs.reduce(_ ++ _).toSet } if (log.isDebugEnabled) { - logDebug(s"live jobs : ${trackListener.liveJobToStages.keySet}") - logDebug(s"stages in jobs : ${trackListener.liveJobToStages.values.flatten}") - logDebug(s"stages : ${trackListener.stageToTasks.keySet}") - logDebug(s"tasks in stages : ${trackListener.stageToTasks.values.flatten}") + logDebug(s"live jobs : ${liveJobToStages.keySet}") + logDebug(s"stages in jobs : ${liveJobToStages.values.flatten}") + logDebug(s"stages : ${stageToTasks.keySet}") + logDebug(s"tasks in stages : ${stageToTasks.values.flatten}") + logDebug(s"RDDs in stages : ${stageToRDDs.values.flatten}") } override def filterStageCompleted(event: SparkListenerStageCompleted): Option[Boolean] = { - Some(trackListener.stageToTasks.contains(event.stageInfo.stageId)) + Some(stageToTasks.contains(event.stageInfo.stageId)) } override def filterStageSubmitted(event: SparkListenerStageSubmitted): Option[Boolean] = { - Some(trackListener.stageToTasks.contains(event.stageInfo.stageId)) + Some(stageToTasks.contains(event.stageInfo.stageId)) } override def filterTaskStart(event: SparkListenerTaskStart): Option[Boolean] = { @@ -98,11 +113,11 @@ class BasicEventFilter(trackListener: BasicEventFilterBuilder) extends EventFilt } override def filterJobStart(event: SparkListenerJobStart): Option[Boolean] = { - Some(trackListener.liveJobToStages.contains(event.jobId)) + Some(liveJobToStages.contains(event.jobId)) } override def filterJobEnd(event: SparkListenerJobEnd): Option[Boolean] = { - Some(trackListener.liveJobToStages.contains(event.jobId)) + Some(liveJobToStages.contains(event.jobId)) } override def filterUnpersistRDD(event: SparkListenerUnpersistRDD): Option[Boolean] = { @@ -111,29 +126,39 @@ class BasicEventFilter(trackListener: BasicEventFilterBuilder) extends EventFilt override def filterStageExecutorMetrics( event: SparkListenerStageExecutorMetrics): Option[Boolean] = { - Some(trackListener.liveExecutors.contains(event.execId)) + Some(liveExecutors.contains(event.execId)) } override def filterExecutorAdded(event: SparkListenerExecutorAdded): Option[Boolean] = { - Some(trackListener.liveExecutors.contains(event.executorId)) + Some(liveExecutors.contains(event.executorId)) } override def filterExecutorRemoved(event: SparkListenerExecutorRemoved): Option[Boolean] = { - Some(trackListener.liveExecutors.contains(event.executorId)) + Some(liveExecutors.contains(event.executorId)) } override def filterExecutorBlacklisted( event: SparkListenerExecutorBlacklisted): Option[Boolean] = { - Some(trackListener.liveExecutors.contains(event.executorId)) + Some(liveExecutors.contains(event.executorId)) } override def filterExecutorUnblacklisted( event: SparkListenerExecutorUnblacklisted): Option[Boolean] = { - Some(trackListener.liveExecutors.contains(event.executorId)) + Some(liveExecutors.contains(event.executorId)) } override def filterSpeculativeTaskSubmitted( event: SparkListenerSpeculativeTaskSubmitted): Option[Boolean] = { - Some(trackListener.stageToTasks.contains(event.stageId)) + Some(stageToTasks.contains(event.stageId)) + } +} + +private[spark] object BasicEventFilter { + def apply(builder: BasicEventFilterBuilder): BasicEventFilter = { + new BasicEventFilter( + builder.liveJobToStages, + builder.stageToTasks, + builder.stageToRDDs, + builder.liveExecutors) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index 291170941b2ca..cdf38fd5669aa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -35,6 +35,7 @@ import org.apache.spark.internal.config.EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN import org.apache.spark.scheduler._ import org.apache.spark.util.{JsonProtocol, Utils} +// FIXME: UTs class EventLogFileCompactor( sparkConf: SparkConf, hadoopConf: Configuration, @@ -111,6 +112,7 @@ class EventLogFileCompactor( } } +// FIXME: UTs class FilteredEventLogFileRewriter( sparkConf: SparkConf, hadoopConf: Configuration, diff --git a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala new file mode 100644 index 0000000000000..ca570c257c025 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala @@ -0,0 +1,236 @@ +/* + * 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.spark.{SparkFunSuite, Success, TaskResultLost, TaskState} +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.scheduler._ +import org.apache.spark.status.ListenerEventsTestHelper + +class BasicEventFilterBuilderSuite extends SparkFunSuite { + import ListenerEventsTestHelper._ + + override protected def beforeEach(): Unit = { + ListenerEventsTestHelper.reset() + } + + test("track live jobs") { + var time = 0L + + val listener = new BasicEventFilterBuilder + listener.onOtherEvent(SparkListenerLogStart("TestSparkVersion")) + + // Start the application. + time += 1 + listener.onApplicationStart(SparkListenerApplicationStart( + "name", + Some("id"), + time, + "user", + Some("attempt"), + None)) + + // Start a couple of executors. + time += 1 + val execIds = Array("1", "2") + execIds.foreach { id => + listener.onExecutorAdded(createExecutorAddedEvent(id, time)) + } + + // Start a job with 2 stages / 4 tasks each + time += 1 + + val rddsForStage1 = createRdds(2) + val rddsForStage2 = createRdds(2) + + val stage1 = createStage(rddsForStage1, Nil) + val stage2 = createStage(rddsForStage2, Seq(stage1.stageId)) + val stages = Seq(stage1, stage2) + + val jobProps = createJobProps() + listener.onJobStart(SparkListenerJobStart(1, time, stages, jobProps)) + + // Submit stage 1 + time += 1 + stages.head.submissionTime = Some(time) + listener.onStageSubmitted(SparkListenerStageSubmitted(stages.head, jobProps)) + + // Start tasks from stage 1 + time += 1 + + val s1Tasks = ListenerEventsTestHelper.createTasks(4, execIds, time) + s1Tasks.foreach { task => + listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, + stages.head.attemptNumber(), task)) + } + + // Fail one of the tasks, re-start it. + time += 1 + s1Tasks.head.markFinished(TaskState.FAILED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber, + "taskType", TaskResultLost, s1Tasks.head, new ExecutorMetrics, null)) + + time += 1 + val reattempt = createTaskWithNewAttempt(s1Tasks.head, time) + listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptNumber, + reattempt)) + + // Succeed all tasks in stage 1. + val pending = s1Tasks.drop(1) ++ Seq(reattempt) + + val s1Metrics = TaskMetrics.empty + s1Metrics.setExecutorCpuTime(2L) + s1Metrics.setExecutorRunTime(4L) + + time += 1 + pending.foreach { task => + task.markFinished(TaskState.FINISHED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber, + "taskType", Success, task, new ExecutorMetrics, s1Metrics)) + } + + // End stage 1. + time += 1 + stages.head.completionTime = Some(time) + listener.onStageCompleted(SparkListenerStageCompleted(stages.head)) + + assert(listener.liveJobToStages.keys.toSeq === Seq(1)) + assert(listener.liveJobToStages(1) === Seq(0, 1)) + assert(listener.stageToRDDs.keys.toSeq === Seq(0)) + assert(listener.stageToRDDs(0) === rddsForStage1.map(_.id)) + // stage 1 not yet submitted + assert(listener.stageToTasks.keys.toSeq === Seq(0)) + assert(listener.stageToTasks(0) === (s1Tasks ++ Seq(reattempt)).map(_.taskId).toSet) + + // Submit stage 2. + time += 1 + stages.last.submissionTime = Some(time) + listener.onStageSubmitted(SparkListenerStageSubmitted(stages.last, jobProps)) + + // Start and fail all tasks of stage 2. + time += 1 + val s2Tasks = createTasks(4, execIds, time) + s2Tasks.foreach { task => + listener.onTaskStart(SparkListenerTaskStart(stages.last.stageId, + stages.last.attemptNumber, + task)) + } + + time += 1 + s2Tasks.foreach { task => + task.markFinished(TaskState.FAILED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stages.last.stageId, stages.last.attemptNumber, + "taskType", TaskResultLost, task, new ExecutorMetrics, null)) + } + + // Fail stage 2. + time += 1 + stages.last.completionTime = Some(time) + stages.last.failureReason = Some("uh oh") + listener.onStageCompleted(SparkListenerStageCompleted(stages.last)) + + // - Re-submit stage 2, all tasks, and succeed them and the stage. + val oldS2 = stages.last + val newS2 = new StageInfo(oldS2.stageId, oldS2.attemptNumber + 1, oldS2.name, oldS2.numTasks, + oldS2.rddInfos, oldS2.parentIds, oldS2.details, oldS2.taskMetrics) + + time += 1 + newS2.submissionTime = Some(time) + listener.onStageSubmitted(SparkListenerStageSubmitted(newS2, jobProps)) + + val newS2Tasks = createTasks(4, execIds, time) + + newS2Tasks.foreach { task => + listener.onTaskStart(SparkListenerTaskStart(newS2.stageId, newS2.attemptNumber, task)) + } + + time += 1 + newS2Tasks.foreach { task => + task.markFinished(TaskState.FINISHED, time) + listener.onTaskEnd(SparkListenerTaskEnd(newS2.stageId, newS2.attemptNumber, "taskType", + Success, task, new ExecutorMetrics, null)) + } + + time += 1 + newS2.completionTime = Some(time) + listener.onStageCompleted(SparkListenerStageCompleted(newS2)) + + assert(listener.liveJobToStages.keys.toSeq === Seq(1)) + assert(listener.liveJobToStages(1) === Seq(0, 1)) + assert(listener.stageToRDDs.keys === Set(0, 1)) + assert(listener.stageToRDDs(0) === rddsForStage1.map(_.id)) + assert(listener.stageToRDDs(1) === rddsForStage2.map(_.id)) + assert(listener.stageToTasks.keys.toSet === Set(0, 1)) + // stage 0 is finished but it stores the information regarding stage + assert(listener.stageToTasks(0) === (s1Tasks ++ Seq(reattempt)).map(_.taskId).toSet) + // stage 1 is newly added + assert(listener.stageToTasks(1) === (s2Tasks ++ newS2Tasks).map(_.taskId).toSet) + + // Start next job. + time += 1 + + val rddsForStage3 = createRdds(2) + val rddsForStage4 = createRdds(2) + + val stage3 = createStage(rddsForStage3, Nil) + val stage4 = createStage(rddsForStage4, Seq(stage3.stageId)) + val stagesForJob2 = Seq(stage3, stage4) + + listener.onJobStart(SparkListenerJobStart(2, time, stagesForJob2, jobProps)) + + // End job 1. + time += 1 + listener.onJobEnd(SparkListenerJobEnd(1, time, JobSucceeded)) + + // everything related to job 1 should be cleaned up, but not for job 2 + assert(listener.liveJobToStages.keys.toSet === Set(2)) + assert(listener.stageToRDDs.isEmpty) + // stageToTasks has no information for job 2, as no task has been started + assert(listener.stageToTasks.isEmpty) + } + + test("track live executors") { + var time = 0L + + val listener = new BasicEventFilterBuilder + listener.onOtherEvent(SparkListenerLogStart("TestSparkVersion")) + + // Start the application. + time += 1 + listener.onApplicationStart(SparkListenerApplicationStart( + "name", + Some("id"), + time, + "user", + Some("attempt"), + None)) + + // Start a couple of executors. + time += 1 + val execIds = (1 to 3).map(_.toString) + execIds.foreach { id => + listener.onExecutorAdded(createExecutorAddedEvent(id, time)) + } + + // End one of executors. + time += 1 + listener.onExecutorRemoved(createExecutorRemovedEvent(execIds.head, time)) + + assert(listener.liveExecutors === execIds.drop(1).toSet) + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala new file mode 100644 index 0000000000000..8395b2fb180ff --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala @@ -0,0 +1,170 @@ +/* + * 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.spark.{storage, SparkFunSuite, Success, TaskState} +import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.scheduler._ +import org.apache.spark.status.ListenerEventsTestHelper +import org.apache.spark.storage.{BlockManagerId, RDDBlockId, StorageLevel} + +class BasicEventFilterSuite extends SparkFunSuite { + import ListenerEventsTestHelper._ + + test("filter out events for finished jobs") { + // assume finished job 1 with stage 1, tasks (1, 2), rdds (1, 2) + // live job 2 with stages 2, tasks (3, 4), rdds (3, 4) + val liveJobToStages: Map[Int, Seq[Int]] = Map(2 -> Seq(2, 3)) + val stageToTasks: Map[Int, Set[Long]] = Map(2 -> Set(3, 4), 3 -> Set(5, 6)) + val stageToRDDs: Map[Int, Seq[Int]] = Map(2 -> Seq(3, 4), 3 -> Seq(5, 6)) + val liveExecutors: Set[String] = Set("1", "2") + + val filter = new BasicEventFilter(liveJobToStages, stageToTasks, stageToRDDs, liveExecutors) + + // Verifying with finished job 1 + val rddsForStage1 = createRddsWithId(1 to 2) + val stage1 = createStage(1, rddsForStage1, Nil) + val tasksForStage1 = createTasks(Seq(1L, 2L), liveExecutors.toArray, 0) + tasksForStage1.foreach { task => task.markFinished(TaskState.FINISHED, 5) } + + val jobStartEventForJob1 = SparkListenerJobStart(1, 0, Seq(stage1)) + val jobEndEventForJob1 = SparkListenerJobEnd(1, 0, JobSucceeded) + val stageSubmittedEventsForJob1 = SparkListenerStageSubmitted(stage1) + val stageCompletedEventsForJob1 = SparkListenerStageCompleted(stage1) + val unpersistRDDEventsForJob1 = (1 to 2).map(SparkListenerUnpersistRDD) + + // job events for finished job should be filtered out + assert(filter.filterJobStart(jobStartEventForJob1) === Some(false)) + assert(filter.filterJobEnd(jobEndEventForJob1) === Some(false)) + + // stage events for finished job should be filtered out + // NOTE: it doesn't filter out stage events which are also related to the executor + assert(filter.filterStageSubmitted(stageSubmittedEventsForJob1) === Some(false)) + assert(filter.filterStageCompleted(stageCompletedEventsForJob1) === Some(false)) + unpersistRDDEventsForJob1.foreach { event => + assert(filter.filterUnpersistRDD(event) === Some(false)) + } + + val taskSpeculativeTaskSubmittedEvent = SparkListenerSpeculativeTaskSubmitted(stage1.stageId, + stageAttemptId = 1) + assert(filter.filterSpeculativeTaskSubmitted(taskSpeculativeTaskSubmittedEvent) === Some(false)) + + // task events for finished job should be filtered out + tasksForStage1.foreach { task => + val taskStartEvent = SparkListenerTaskStart(stage1.stageId, 0, task) + assert(filter.filterTaskStart(taskStartEvent) === Some(false)) + + val taskGettingResultEvent = SparkListenerTaskGettingResult(task) + assert(filter.filterTaskGettingResult(taskGettingResultEvent) === Some(false)) + + val taskEndEvent = SparkListenerTaskEnd(stage1.stageId, 0, "taskType", + Success, task, new ExecutorMetrics, null) + assert(filter.filterTaskEnd(taskEndEvent) === Some(false)) + } + + // Verifying with live job 2 + val rddsForStage2 = createRddsWithId(3 to 4) + val stage2 = createStage(2, rddsForStage2, Nil) + val tasksForStage2 = createTasks(Seq(3L, 4L), liveExecutors.toArray, 0) + tasksForStage1.foreach { task => task.markFinished(TaskState.FINISHED, 5) } + + val jobStartEventForJob2 = SparkListenerJobStart(2, 0, Seq(stage2)) + val stageSubmittedEventsForJob2 = SparkListenerStageSubmitted(stage2) + val stageCompletedEventsForJob2 = SparkListenerStageCompleted(stage2) + val unpersistRDDEventsForJob2 = rddsForStage2.map { rdd => SparkListenerUnpersistRDD(rdd.id) } + + // job events for live job should be filtered in + assert(filter.filterJobStart(jobStartEventForJob2) === Some(true)) + + // stage events for live job should be filtered in + assert(filter.filterStageSubmitted(stageSubmittedEventsForJob2) === Some(true)) + assert(filter.filterStageCompleted(stageCompletedEventsForJob2) === Some(true)) + unpersistRDDEventsForJob2.foreach { event => + assert(filter.filterUnpersistRDD(event) === Some(true)) + } + + val taskSpeculativeTaskSubmittedEvent2 = SparkListenerSpeculativeTaskSubmitted(stage2.stageId, + stageAttemptId = 1) + assert(filter.filterSpeculativeTaskSubmitted(taskSpeculativeTaskSubmittedEvent2) === Some(true)) + + // task events for live job should be filtered in + tasksForStage2.foreach { task => + val taskStartEvent = SparkListenerTaskStart(stage2.stageId, 0, task) + assert(filter.filterTaskStart(taskStartEvent) === Some(true)) + + val taskGettingResultEvent = SparkListenerTaskGettingResult(task) + assert(filter.filterTaskGettingResult(taskGettingResultEvent) === Some(true)) + + val taskEndEvent = SparkListenerTaskEnd(stage1.stageId, 0, "taskType", + Success, task, new ExecutorMetrics, null) + assert(filter.filterTaskEnd(taskEndEvent) === Some(true)) + } + } + + test("filter out events for dead executors") { + // assume executor 1 was dead, and live executor 2 is available + val liveExecutors: Set[String] = Set("2") + + val filter = new BasicEventFilter(Map.empty, Map.empty, Map.empty, liveExecutors) + + // events for dead executor should be filtered out + assert(filter.filterExecutorAdded(createExecutorAddedEvent(1)) === Some(false)) + // though the name of event is stage executor metrics, AppStatusListener only deals with + // live executors + assert(filter.filterStageExecutorMetrics( + SparkListenerStageExecutorMetrics(1.toString, 0, 0, new ExecutorMetrics)) === + Some(false)) + assert(filter.filterExecutorBlacklisted(SparkListenerExecutorBlacklisted(0, 1.toString, 1)) + === Some(false)) + assert(filter.filterExecutorUnblacklisted(SparkListenerExecutorUnblacklisted(0, 1.toString)) + === Some(false)) + assert(filter.filterExecutorRemoved(createExecutorRemovedEvent(1)) === Some(false)) + + // events for live executor should be filtered in + assert(filter.filterExecutorAdded(createExecutorAddedEvent(2)) === Some(true)) + assert(filter.filterStageExecutorMetrics( + SparkListenerStageExecutorMetrics(2.toString, 0, 0, new ExecutorMetrics)) === + Some(true)) + assert(filter.filterExecutorBlacklisted(SparkListenerExecutorBlacklisted(0, 2.toString, 1)) + === Some(true)) + assert(filter.filterExecutorUnblacklisted(SparkListenerExecutorUnblacklisted(0, 2.toString)) + === Some(true)) + assert(filter.filterExecutorRemoved(createExecutorRemovedEvent(2)) === Some(true)) + } + + test("other events should be left to other filters") { + def assertNone(predicate: => Option[Boolean]): Unit = { + assert(predicate === None) + } + + val filter = new BasicEventFilter(Map.empty, Map.empty, Map.empty, Set.empty) + + assertNone(filter.filterEnvironmentUpdate(SparkListenerEnvironmentUpdate(Map.empty))) + assertNone(filter.filterApplicationStart( + SparkListenerApplicationStart("1", Some("1"), 0, "user", None))) + assertNone(filter.filterApplicationEnd(SparkListenerApplicationEnd(1))) + val bmId = BlockManagerId("1", "host1", 1) + assertNone(filter.filterBlockManagerAdded(SparkListenerBlockManagerAdded(0, bmId, 1))) + assertNone(filter.filterBlockManagerRemoved(SparkListenerBlockManagerRemoved(1, bmId))) + assertNone(filter.filterBlockUpdated(SparkListenerBlockUpdated( + storage.BlockUpdatedInfo(bmId, RDDBlockId(1, 1), StorageLevel.DISK_ONLY, 0, 10)))) + assertNone(filter.filterNodeBlacklisted(SparkListenerNodeBlacklisted(0, "host1", 1))) + assertNone(filter.filterNodeUnblacklisted(SparkListenerNodeUnblacklisted(0, "host1"))) + assertNone(filter.filterOtherEvent(SparkListenerLogStart("testVersion"))) + } +} diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index a289dddbdc9e6..10ff95ad935cb 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -37,6 +37,7 @@ import org.apache.spark.storage._ import org.apache.spark.util.Utils class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { + import ListenerEventsTestHelper._ private val conf = new SparkConf() .set(LIVE_ENTITY_UPDATE_PERIOD, 0L) @@ -1694,40 +1695,4 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { def blockId: BlockId = RDDBlockId(rddId, partId) } - - /** Create a stage submitted event for the specified stage Id. */ - private def createStageSubmittedEvent(stageId: Int) = { - SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0, - Seq.empty, Seq.empty, "details")) - } - - /** Create a stage completed event for the specified stage Id. */ - private def createStageCompletedEvent(stageId: Int) = { - SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0, - Seq.empty, Seq.empty, "details")) - } - - /** Create an executor added event for the specified executor Id. */ - private def createExecutorAddedEvent(executorId: Int) = { - SparkListenerExecutorAdded(0L, executorId.toString, - new ExecutorInfo("host1", 1, Map.empty, Map.empty)) - } - - /** Create an executor added event for the specified executor Id. */ - private def createExecutorRemovedEvent(executorId: Int) = { - SparkListenerExecutorRemoved(10L, executorId.toString, "test") - } - - /** Create an executor metrics update event, with the specified executor metrics values. */ - private def createExecutorMetricsUpdateEvent( - stageId: Int, - executorId: Int, - executorMetrics: Array[Long]): SparkListenerExecutorMetricsUpdate = { - val taskMetrics = TaskMetrics.empty - taskMetrics.incDiskBytesSpilled(111) - taskMetrics.incMemoryBytesSpilled(222) - val accum = Array((333L, 1, 1, taskMetrics.accumulators().map(AccumulatorSuite.makeInfo))) - val executorUpdates = Map((stageId, 0) -> new ExecutorMetrics(executorMetrics)) - SparkListenerExecutorMetricsUpdate(executorId.toString, accum, executorUpdates) - } } diff --git a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala new file mode 100644 index 0000000000000..cf96c3a52d6f7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala @@ -0,0 +1,153 @@ +/* + * 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.status + +import java.util.Properties + +import scala.collection.immutable.Map +import org.apache.spark.{AccumulatorSuite, SparkContext} +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.scheduler.{SparkListenerExecutorAdded, SparkListenerExecutorMetricsUpdate, SparkListenerExecutorRemoved, SparkListenerStageCompleted, SparkListenerStageSubmitted, SparkListenerTaskStart, StageInfo, TaskInfo, TaskLocality} +import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.storage.{RDDInfo, StorageLevel} + +object ListenerEventsTestHelper { + + private var taskIdTracker = -1L + private var rddIdTracker = -1 + private var stageIdTracker = -1 + + def reset(): Unit = { + taskIdTracker = -1L + rddIdTracker = -1 + stageIdTracker = -1 + } + + def createJobProps(): Properties = { + val jobProps = new Properties() + jobProps.setProperty(SparkContext.SPARK_JOB_DESCRIPTION, "jobDescription") + jobProps.setProperty(SparkContext.SPARK_JOB_GROUP_ID, "jobGroup") + jobProps.setProperty(SparkContext.SPARK_SCHEDULER_POOL, "schedPool") + jobProps + } + + def createRddsWithId(ids: Seq[Int]): Seq[RDDInfo] = { + ids.map { rddId => + new RDDInfo(rddId, s"rdd${rddId}", 2, StorageLevel.NONE, false, Nil) + } + } + + def createRdds(count: Int): Seq[RDDInfo] = { + (1 to count).map { _ => + val rddId = nextRddId() + new RDDInfo(rddId, s"rdd${rddId}", 2, StorageLevel.NONE, false, Nil) + } + } + + def createStage(id: Int, rdds: Seq[RDDInfo], parentIds: Seq[Int]): StageInfo = { + new StageInfo(id, 0, s"stage${id}", 4, rdds, parentIds, s"details${id}") + } + + def createStage(rdds: Seq[RDDInfo], parentIds: Seq[Int]): StageInfo = { + createStage(nextStageId(), rdds, parentIds) + } + + def createTasks(ids: Seq[Long], execs: Array[String], time: Long): Seq[TaskInfo] = { + ids.zipWithIndex.map { case (id, idx) => + val exec = execs(idx % execs.length) + new TaskInfo(id, idx, 1, time, exec, s"$exec.example.com", + TaskLocality.PROCESS_LOCAL, idx % 2 == 0) + } + } + + def createTasks(count: Int, execs: Array[String], time: Long): Seq[TaskInfo] = { + createTasks((1 to count).map { _ => nextTaskId() }, execs, time) + } + + def createTaskWithNewAttempt(orig: TaskInfo, time: Long): TaskInfo = { + // Task reattempts have a different ID, but the same index as the original. + new TaskInfo(nextTaskId(), orig.index, orig.attemptNumber + 1, time, orig.executorId, + s"${orig.executorId}.example.com", TaskLocality.PROCESS_LOCAL, orig.speculative) + } + + def createTaskStartEvent( + taskInfo: TaskInfo, + stageId: Int, + attemptId: Int): SparkListenerTaskStart = { + SparkListenerTaskStart(stageId, attemptId, taskInfo) + } + + /** Create a stage submitted event for the specified stage Id. */ + def createStageSubmittedEvent(stageId: Int): SparkListenerStageSubmitted = { + SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0, + Seq.empty, Seq.empty, "details")) + } + + /** Create a stage completed event for the specified stage Id. */ + def createStageCompletedEvent(stageId: Int): SparkListenerStageCompleted = { + SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0, + Seq.empty, Seq.empty, "details")) + } + + def createExecutorAddedEvent(executorId: Int): SparkListenerExecutorAdded = { + createExecutorAddedEvent(executorId.toString, 0) + } + + /** Create an executor added event for the specified executor Id. */ + def createExecutorAddedEvent(executorId: String, time: Long): SparkListenerExecutorAdded = { + SparkListenerExecutorAdded(time, executorId, + new ExecutorInfo("host1", 1, Map.empty, Map.empty)) + } + + def createExecutorRemovedEvent(executorId: Int): SparkListenerExecutorRemoved = { + createExecutorRemovedEvent(executorId.toString, 0) + } + + /** Create an executor added event for the specified executor Id. */ + def createExecutorRemovedEvent(executorId: String, time: Long): SparkListenerExecutorRemoved = { + SparkListenerExecutorRemoved(time, executorId, "test") + } + + /** Create an executor metrics update event, with the specified executor metrics values. */ + def createExecutorMetricsUpdateEvent( + stageId: Int, + executorId: Int, + executorMetrics: Array[Long]): SparkListenerExecutorMetricsUpdate = { + val taskMetrics = TaskMetrics.empty + taskMetrics.incDiskBytesSpilled(111) + taskMetrics.incMemoryBytesSpilled(222) + val accum = Array((333L, 1, 1, taskMetrics.accumulators().map(AccumulatorSuite.makeInfo))) + val executorUpdates = Map((stageId, 0) -> new ExecutorMetrics(executorMetrics)) + SparkListenerExecutorMetricsUpdate(executorId.toString, accum, executorUpdates) + } + + private def nextTaskId(): Long = { + taskIdTracker += 1 + taskIdTracker + } + + private def nextRddId(): Int = { + rddIdTracker += 1 + rddIdTracker + } + + private def nextStageId(): Int = { + stageIdTracker += 1 + stageIdTracker + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala index e54b241bb84d9..19ffae8fca17c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.ui._ import org.apache.spark.sql.streaming.StreamingQueryListener +// FIXME: UTs class SQLEventFilterBuilder extends SparkListener with EventFilterBuilder { val liveExecutionToJobs = new mutable.HashMap[Long, mutable.Set[Int]] val jobToStages = new mutable.HashMap[Int, Seq[Int]] @@ -80,6 +81,7 @@ class SQLEventFilterBuilder extends SparkListener with EventFilterBuilder { } } +// FIXME: UTs class SQLLiveEntitiesEventFilter(trackListener: SQLEventFilterBuilder) extends EventFilter with Logging { From 916a4583ea4246b710d01728d67083900c4f6987 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 13 Nov 2019 15:02:59 +0900 Subject: [PATCH 05/28] Address UTs for SQL event filter (builder) --- .../history/BasicEventFilterBuilder.scala | 6 +- .../history/SQLEventFilterBuilder.scala | 132 +++++++++----- .../history/SQLEventFilterBuilderSuite.scala | 161 ++++++++++++++++++ .../SQLLiveEntitiesEventFilterSuite.scala | 135 +++++++++++++++ 4 files changed, 390 insertions(+), 44 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala index ba5b192740844..55c50b92cbd7d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala @@ -40,10 +40,8 @@ private[spark] class BasicEventFilterBuilder extends SparkListener with EventFil override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { val stages = _liveJobToStages.getOrElse(jobEnd.jobId, Seq.empty[Int]) _liveJobToStages -= jobEnd.jobId - stages.foreach { stage => - _stageToTasks -= stage - _stageToRDDs -= stage - } + _stageToTasks --= stages + _stageToRDDs --= stages } override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala index 19ffae8fca17c..07795313002e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala @@ -26,11 +26,17 @@ import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.ui._ import org.apache.spark.sql.streaming.StreamingQueryListener -// FIXME: UTs -class SQLEventFilterBuilder extends SparkListener with EventFilterBuilder { - val liveExecutionToJobs = new mutable.HashMap[Long, mutable.Set[Int]] - val jobToStages = new mutable.HashMap[Int, Seq[Int]] - val stageToTasks = new mutable.HashMap[Int, mutable.Set[Long]] +private[spark] class SQLEventFilterBuilder extends SparkListener with EventFilterBuilder { + private val _liveExecutionToJobs = new mutable.HashMap[Long, mutable.Set[Int]] + private val _jobToStages = new mutable.HashMap[Int, Seq[Int]] + private val _stageToTasks = new mutable.HashMap[Int, mutable.Set[Long]] + private val _stageToRDDs = new mutable.HashMap[Int, Seq[Int]] + private val stages = new mutable.HashSet[Int] + + def liveExecutionToJobs: Map[Long, Set[Int]] = _liveExecutionToJobs.mapValues(_.toSet).toMap + def jobToStages: Map[Int, Seq[Int]] = _jobToStages.toMap + def stageToTasks: Map[Int, Set[Long]] = _stageToTasks.mapValues(_.toSet).toMap + def stageToRDDs: Map[Int, Seq[Int]] = _stageToRDDs.toMap override def onJobStart(jobStart: SparkListenerJobStart): Unit = { val executionIdString = jobStart.properties.getProperty(SQLExecution.EXECUTION_ID_KEY) @@ -42,17 +48,26 @@ class SQLEventFilterBuilder extends SparkListener with EventFilterBuilder { val executionId = executionIdString.toLong val jobId = jobStart.jobId - val jobsForExecution = liveExecutionToJobs.getOrElseUpdate(executionId, + val jobsForExecution = _liveExecutionToJobs.getOrElseUpdate(executionId, mutable.HashSet[Int]()) jobsForExecution += jobId - jobToStages += jobStart.jobId -> jobStart.stageIds - jobStart.stageIds.foreach { stageId => stageToTasks += stageId -> mutable.HashSet[Long]() } + _jobToStages += jobStart.jobId -> jobStart.stageIds + stages ++= jobStart.stageIds + jobStart.stageIds.foreach { stageId => _stageToTasks += stageId -> mutable.HashSet[Long]() } + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { + val stageId = stageSubmitted.stageInfo.stageId + if (stages.contains(stageId)) { + val rddInfos = stageSubmitted.stageInfo.rddInfos + _stageToRDDs += stageId -> rddInfos.map(_.id) + } } override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { - if (stageToTasks.contains(taskStart.stageId)) { - val curTasks = stageToTasks(taskStart.stageId) + if (_stageToTasks.contains(taskStart.stageId)) { + val curTasks = _stageToTasks(taskStart.stageId) curTasks += taskStart.taskInfo.taskId } } @@ -64,76 +79,95 @@ class SQLEventFilterBuilder extends SparkListener with EventFilterBuilder { } private def onExecutionStart(event: SparkListenerSQLExecutionStart): Unit = { - liveExecutionToJobs += event.executionId -> mutable.HashSet[Int]() + _liveExecutionToJobs += event.executionId -> mutable.HashSet[Int]() } private def onExecutionEnd(event: SparkListenerSQLExecutionEnd): Unit = { - val jobs = liveExecutionToJobs.getOrElse(event.executionId, mutable.HashSet[Int]()) - liveExecutionToJobs -= event.executionId + val jobs = _liveExecutionToJobs.getOrElse(event.executionId, mutable.HashSet[Int]()) + _liveExecutionToJobs -= event.executionId - val stages = jobToStages.filter(kv => jobs.contains(kv._1)).values.flatten - jobToStages --= jobs - stageToTasks --= stages + val stagesToDrop = _jobToStages.filter(kv => jobs.contains(kv._1)).values.flatten + _jobToStages --= jobs + stages --= stagesToDrop + _stageToTasks --= stagesToDrop + _stageToRDDs --= stagesToDrop } override def createFilter(): EventFilter = { - new SQLLiveEntitiesEventFilter(this) + SQLLiveEntitiesEventFilter(this) } } -// FIXME: UTs -class SQLLiveEntitiesEventFilter(trackListener: SQLEventFilterBuilder) - extends EventFilter with Logging { +private[spark] class SQLLiveEntitiesEventFilter( + liveExecutionToJobs: Map[Long, Set[Int]], + jobToStages: Map[Int, Seq[Int]], + stageToTasks: Map[Int, Set[Long]], + stageToRDDs: Map[Int, Seq[Int]]) extends EventFilter with Logging { - private val liveTasks: Set[Long] = trackListener.stageToTasks.values match { + private val liveTasks: Set[Long] = stageToTasks.values match { case xs if xs.isEmpty => Set.empty[Long] case xs => xs.reduce(_ ++ _).toSet } + private val liveRDDs: Set[Int] = stageToRDDs.values match { + case xs if xs.isEmpty => Set.empty[Int] + case xs => xs.reduce(_ ++ _).toSet + } + if (log.isDebugEnabled) { - logDebug(s"live executions : ${trackListener.liveExecutionToJobs.keySet}") - logDebug(s"jobs in live executions : ${trackListener.liveExecutionToJobs.values.flatten}") - logDebug(s"jobs : ${trackListener.jobToStages.keySet}") - logDebug(s"stages in jobs : ${trackListener.jobToStages.values.flatten}") - logDebug(s"stages : ${trackListener.stageToTasks.keySet}") - logDebug(s"tasks in stages : ${trackListener.stageToTasks.values.flatten}") + logDebug(s"live executions : ${liveExecutionToJobs.keySet}") + logDebug(s"jobs in live executions : ${liveExecutionToJobs.values.flatten}") + logDebug(s"jobs : ${jobToStages.keySet}") + logDebug(s"stages in jobs : ${jobToStages.values.flatten}") + logDebug(s"stages : ${stageToTasks.keySet}") + logDebug(s"tasks in stages : ${stageToTasks.values.flatten}") + logDebug(s"RDDs in stages : ${stageToRDDs.values.flatten}") } override def filterStageCompleted(event: SparkListenerStageCompleted): Option[Boolean] = { - Some(trackListener.stageToTasks.contains(event.stageInfo.stageId)) + trueOrNone(stageToTasks.contains(event.stageInfo.stageId)) } override def filterStageSubmitted(event: SparkListenerStageSubmitted): Option[Boolean] = { - Some(trackListener.stageToTasks.contains(event.stageInfo.stageId)) + trueOrNone(stageToTasks.contains(event.stageInfo.stageId)) } override def filterTaskStart(event: SparkListenerTaskStart): Option[Boolean] = { - Some(liveTasks.contains(event.taskInfo.taskId)) + trueOrNone(liveTasks.contains(event.taskInfo.taskId)) } override def filterTaskGettingResult(event: SparkListenerTaskGettingResult): Option[Boolean] = { - Some(liveTasks.contains(event.taskInfo.taskId)) + trueOrNone(liveTasks.contains(event.taskInfo.taskId)) } override def filterTaskEnd(event: SparkListenerTaskEnd): Option[Boolean] = { - Some(liveTasks.contains(event.taskInfo.taskId)) + trueOrNone(liveTasks.contains(event.taskInfo.taskId)) } override def filterJobStart(event: SparkListenerJobStart): Option[Boolean] = { - Some(trackListener.jobToStages.contains(event.jobId)) + trueOrNone(jobToStages.contains(event.jobId)) } override def filterJobEnd(event: SparkListenerJobEnd): Option[Boolean] = { - Some(trackListener.jobToStages.contains(event.jobId)) + trueOrNone(jobToStages.contains(event.jobId)) + } + + override def filterUnpersistRDD(event: SparkListenerUnpersistRDD): Option[Boolean] = { + trueOrNone(liveRDDs.contains(event.rddId)) } override def filterExecutorMetricsUpdate( event: SparkListenerExecutorMetricsUpdate): Option[Boolean] = { - Some(event.accumUpdates.exists { case (_, stageId, _, _) => - trackListener.stageToTasks.contains(stageId) + trueOrNone(event.accumUpdates.exists { case (_, stageId, _, _) => + stageToTasks.contains(stageId) }) } + override def filterSpeculativeTaskSubmitted( + event: SparkListenerSpeculativeTaskSubmitted): Option[Boolean] = { + trueOrNone(stageToTasks.contains(event.stageId)) + } + override def filterOtherEvent(event: SparkListenerEvent): Option[Boolean] = event match { case e: SparkListenerSQLExecutionStart => filterExecutionStart(e) case e: SparkListenerSQLAdaptiveExecutionUpdate => filterAdaptiveExecutionUpdate(e) @@ -146,19 +180,37 @@ class SQLLiveEntitiesEventFilter(trackListener: SQLEventFilterBuilder) } def filterExecutionStart(event: SparkListenerSQLExecutionStart): Option[Boolean] = { - Some(trackListener.liveExecutionToJobs.contains(event.executionId)) + Some(liveExecutionToJobs.contains(event.executionId)) } def filterAdaptiveExecutionUpdate( event: SparkListenerSQLAdaptiveExecutionUpdate): Option[Boolean] = { - Some(trackListener.liveExecutionToJobs.contains(event.executionId)) + Some(liveExecutionToJobs.contains(event.executionId)) } def filterExecutionEnd(event: SparkListenerSQLExecutionEnd): Option[Boolean] = { - Some(trackListener.liveExecutionToJobs.contains(event.executionId)) + Some(liveExecutionToJobs.contains(event.executionId)) } def filterDriverAccumUpdates(event: SparkListenerDriverAccumUpdates): Option[Boolean] = { - Some(trackListener.liveExecutionToJobs.contains(event.executionId)) + Some(liveExecutionToJobs.contains(event.executionId)) + } + + private def trueOrNone(booleanValue: Boolean): Option[Boolean] = { + if (booleanValue) { + Some(booleanValue) + } else { + None + } + } +} + +private[spark] object SQLLiveEntitiesEventFilter { + def apply(builder: SQLEventFilterBuilder): SQLLiveEntitiesEventFilter = { + new SQLLiveEntitiesEventFilter( + builder.liveExecutionToJobs, + builder.jobToStages, + builder.stageToTasks, + builder.stageToRDDs) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala new file mode 100644 index 0000000000000..7fca1a805f8f9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala @@ -0,0 +1,161 @@ +/* + * 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.sql.execution.history + +import org.apache.spark.{SparkFunSuite, Success, TaskState} +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.scheduler.{JobSucceeded, SparkListenerApplicationStart, SparkListenerJobEnd, SparkListenerJobStart, SparkListenerLogStart, SparkListenerStageCompleted, SparkListenerStageSubmitted, SparkListenerTaskEnd, SparkListenerTaskStart} +import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} +import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart} +import org.apache.spark.status.ListenerEventsTestHelper + +class SQLEventFilterBuilderSuite extends SparkFunSuite { + import ListenerEventsTestHelper._ + + override protected def beforeEach(): Unit = { + ListenerEventsTestHelper.reset() + } + + test("track live SQL executions") { + case class JobInfo( + stageIds: Seq[Int], + stageToTaskIds: Map[Int, Seq[Long]], + stageToRddIds: Map[Int, Seq[Int]]) + + def pushJobEventsWithoutJobEnd( + listener: SQLEventFilterBuilder, + jobId: Int, + execIds: Array[String], + sqlExecId: Option[String], + time: Long): JobInfo = { + // Start a job with 1 stages / 4 tasks each + val rddsForStage = createRdds(2) + val stage = createStage(rddsForStage, Nil) + + val jobProps = createJobProps() + sqlExecId.foreach { id => jobProps.setProperty(SQLExecution.EXECUTION_ID_KEY, id) } + + listener.onJobStart(SparkListenerJobStart(jobId, time, Seq(stage), jobProps)) + + // Submit stage + stage.submissionTime = Some(time) + listener.onStageSubmitted(SparkListenerStageSubmitted(stage, jobProps)) + + // Start tasks from stage + val s1Tasks = ListenerEventsTestHelper.createTasks(4, execIds, time) + s1Tasks.foreach { task => + listener.onTaskStart(SparkListenerTaskStart(stage.stageId, + stage.attemptNumber(), task)) + } + + // Succeed all tasks in stage. + val s1Metrics = TaskMetrics.empty + s1Metrics.setExecutorCpuTime(2L) + s1Metrics.setExecutorRunTime(4L) + + s1Tasks.foreach { task => + task.markFinished(TaskState.FINISHED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, + "taskType", Success, task, new ExecutorMetrics, s1Metrics)) + } + + // End stage. + stage.completionTime = Some(time) + listener.onStageCompleted(SparkListenerStageCompleted(stage)) + + JobInfo(Seq(stage.stageId), Map(stage.stageId -> s1Tasks.map(_.taskId)), + Map(stage.stageId -> rddsForStage.map(_.id))) + } + + var time = 0L + + val listener = new SQLEventFilterBuilder + + listener.onOtherEvent(SparkListenerLogStart("TestSparkVersion")) + + // Start the application. + time += 1 + listener.onApplicationStart(SparkListenerApplicationStart( + "name", + Some("id"), + time, + "user", + Some("attempt"), + None)) + + // Start a couple of executors. + time += 1 + val execIds = Array("1", "2") + execIds.foreach { id => + listener.onExecutorAdded(createExecutorAddedEvent(id, time)) + } + + // Start SQL Execution + listener.onOtherEvent(SparkListenerSQLExecutionStart(1, "desc1", "details1", "plan", + new SparkPlanInfo("node", "str", Seq.empty, Map.empty, Seq.empty), time)) + + time += 1 + + // job 1, 2: coupled with SQL execution 1, finished + val jobInfoForJob1 = pushJobEventsWithoutJobEnd(listener, 1, execIds, Some("1"), time) + listener.onJobEnd(SparkListenerJobEnd(1, time, JobSucceeded)) + + val jobInfoForJob2 = pushJobEventsWithoutJobEnd(listener, 2, execIds, Some("1"), time) + listener.onJobEnd(SparkListenerJobEnd(2, time, JobSucceeded)) + + // job 3: not coupled with SQL execution 1, finished + pushJobEventsWithoutJobEnd(listener, 3, execIds, None, time) + listener.onJobEnd(SparkListenerJobEnd(3, time, JobSucceeded)) + + // job 4: not coupled with SQL execution 1, not finished + pushJobEventsWithoutJobEnd(listener, 4, execIds, None, time) + listener.onJobEnd(SparkListenerJobEnd(4, time, JobSucceeded)) + + assert(listener.liveExecutionToJobs.keys === Set(1)) + assert(listener.liveExecutionToJobs(1) === Set(1, 2)) + + // only SQL executions related jobs are tracked + assert(listener.jobToStages.keys === Set(1, 2)) + assert(listener.jobToStages(1).toSet === jobInfoForJob1.stageIds.toSet) + assert(listener.jobToStages(2).toSet === jobInfoForJob2.stageIds.toSet) + + assert(listener.stageToTasks.keys === + (jobInfoForJob1.stageIds ++ jobInfoForJob2.stageIds).toSet) + listener.stageToTasks.foreach { case (stageId, tasks) => + val expectedTasks = jobInfoForJob1.stageToTaskIds.getOrElse(stageId, + jobInfoForJob2.stageToTaskIds.getOrElse(stageId, null)) + assert(tasks === expectedTasks.toSet) + } + + assert(listener.stageToRDDs.keys === + (jobInfoForJob1.stageIds ++ jobInfoForJob2.stageIds).toSet) + listener.stageToRDDs.foreach { case (stageId, rdds) => + val expectedRDDs = jobInfoForJob1.stageToRddIds.getOrElse(stageId, + jobInfoForJob2.stageToRddIds.getOrElse(stageId, null)) + assert(rdds.toSet === expectedRDDs.toSet) + } + + // End SQL execution + listener.onOtherEvent(SparkListenerSQLExecutionEnd(1, 0)) + + assert(listener.liveExecutionToJobs.isEmpty) + assert(listener.jobToStages.isEmpty) + assert(listener.stageToTasks.isEmpty) + assert(listener.stageToRDDs.isEmpty) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala new file mode 100644 index 0000000000000..85331d032b294 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala @@ -0,0 +1,135 @@ +/* + * 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.sql.execution.history + +import org.apache.spark.{SparkFunSuite, Success, TaskState} +import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.scheduler._ +import org.apache.spark.sql.execution.ui.{SparkListenerDriverAccumUpdates, SparkListenerSQLAdaptiveExecutionUpdate, SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart} +import org.apache.spark.status.ListenerEventsTestHelper.{createRddsWithId, createStage, createTasks} + +class SQLLiveEntitiesEventFilterSuite extends SparkFunSuite { + test("filter in events for jobs related to live SQL execution") { + // assume finished job 1 with stage 1, task (1, 2), rdds (1, 2) and finished sql execution id 1 + // live job 2 with stages 2, tasks (3, 4), rdds (3, 4) and job 2 belongs to the live + // sql execution id 2 + + val liveExecutionToJobs: Map[Long, Set[Int]] = Map(2L -> Set(2)) + val liveJobToStages: Map[Int, Seq[Int]] = Map(2 -> Seq(2, 3)) + val stageToTasks: Map[Int, Set[Long]] = Map(2 -> Set(3L, 4L), 3 -> Set(5L, 6L)) + val stageToRDDs: Map[Int, Seq[Int]] = Map(2 -> Seq(3, 4), 3 -> Seq(5, 6)) + val liveExecutors: Set[String] = Set("1", "2") + + val filter = new SQLLiveEntitiesEventFilter(liveExecutionToJobs, liveJobToStages, + stageToTasks, stageToRDDs) + + // Verifying with finished SQL execution 1 + assert(filter.filterOtherEvent(SparkListenerSQLExecutionStart(1, "description1", "details1", + "plan", null, 0)) === Some(false)) + assert(filter.filterOtherEvent(SparkListenerSQLExecutionEnd(1, 0)) === Some(false)) + assert(filter.filterOtherEvent(SparkListenerSQLAdaptiveExecutionUpdate(1, "plan", null)) + === Some(false)) + assert(filter.filterOtherEvent(SparkListenerDriverAccumUpdates(1, Seq.empty)) === Some(false)) + + // Verifying with finished job 1 + val rddsForStage1 = createRddsWithId(1 to 2) + val stage1 = createStage(1, rddsForStage1, Nil) + val tasksForStage1 = createTasks(Seq(1L, 2L), liveExecutors.toArray, 0) + tasksForStage1.foreach { task => task.markFinished(TaskState.FINISHED, 5) } + + val jobStartEventForJob1 = SparkListenerJobStart(1, 0, Seq(stage1)) + val jobEndEventForJob1 = SparkListenerJobEnd(1, 0, JobSucceeded) + val stageSubmittedEventsForJob1 = SparkListenerStageSubmitted(stage1) + val stageCompletedEventsForJob1 = SparkListenerStageCompleted(stage1) + val unpersistRDDEventsForJob1 = (1 to 2).map(SparkListenerUnpersistRDD) + + // job events for finished job should be considered as "don't know" + assert(filter.filterJobStart(jobStartEventForJob1) === None) + assert(filter.filterJobEnd(jobEndEventForJob1) === None) + + // stage events for finished job should be considered as "don't know" + assert(filter.filterStageSubmitted(stageSubmittedEventsForJob1) === None) + assert(filter.filterStageCompleted(stageCompletedEventsForJob1) === None) + unpersistRDDEventsForJob1.foreach { event => + assert(filter.filterUnpersistRDD(event) === None) + } + + val taskSpeculativeTaskSubmittedEvent = SparkListenerSpeculativeTaskSubmitted(stage1.stageId, + stageAttemptId = 1) + assert(filter.filterSpeculativeTaskSubmitted(taskSpeculativeTaskSubmittedEvent) === None) + + // task events for finished job should be considered as "don't know" + tasksForStage1.foreach { task => + val taskStartEvent = SparkListenerTaskStart(stage1.stageId, 0, task) + assert(filter.filterTaskStart(taskStartEvent) === None) + + val taskGettingResultEvent = SparkListenerTaskGettingResult(task) + assert(filter.filterTaskGettingResult(taskGettingResultEvent) === None) + + val taskEndEvent = SparkListenerTaskEnd(stage1.stageId, 0, "taskType", + Success, task, new ExecutorMetrics, null) + assert(filter.filterTaskEnd(taskEndEvent) === None) + } + + // Verifying with live SQL execution 2 + assert(filter.filterOtherEvent(SparkListenerSQLExecutionStart(2, "description2", "details2", + "plan", null, 0)) === Some(true)) + assert(filter.filterOtherEvent(SparkListenerSQLExecutionEnd(2, 0)) === Some(true)) + assert(filter.filterOtherEvent(SparkListenerSQLAdaptiveExecutionUpdate(2, "plan", null)) + === Some(true)) + assert(filter.filterOtherEvent(SparkListenerDriverAccumUpdates(2, Seq.empty)) === Some(true)) + + // Verifying with live job 2 + val rddsForStage2 = createRddsWithId(3 to 4) + val stage2 = createStage(2, rddsForStage2, Nil) + val tasksForStage2 = createTasks(Seq(3L, 4L), liveExecutors.toArray, 0) + tasksForStage1.foreach { task => task.markFinished(TaskState.FINISHED, 5) } + + val jobStartEventForJob2 = SparkListenerJobStart(2, 0, Seq(stage2)) + val stageSubmittedEventsForJob2 = SparkListenerStageSubmitted(stage2) + val stageCompletedEventsForJob2 = SparkListenerStageCompleted(stage2) + val unpersistRDDEventsForJob2 = rddsForStage2.map { rdd => SparkListenerUnpersistRDD(rdd.id) } + + // job events for live job should be filtered in + assert(filter.filterJobStart(jobStartEventForJob2) === Some(true)) + + // stage events for live job should be filtered in + assert(filter.filterStageSubmitted(stageSubmittedEventsForJob2) === Some(true)) + assert(filter.filterStageCompleted(stageCompletedEventsForJob2) === Some(true)) + unpersistRDDEventsForJob2.foreach { event => + assert(filter.filterUnpersistRDD(event) === Some(true)) + } + + val taskSpeculativeTaskSubmittedEvent2 = SparkListenerSpeculativeTaskSubmitted(stage2.stageId, + stageAttemptId = 1) + assert(filter.filterSpeculativeTaskSubmitted(taskSpeculativeTaskSubmittedEvent2) === Some(true)) + + // task events for live job should be filtered in + tasksForStage2.foreach { task => + val taskStartEvent = SparkListenerTaskStart(stage2.stageId, 0, task) + assert(filter.filterTaskStart(taskStartEvent) === Some(true)) + + val taskGettingResultEvent = SparkListenerTaskGettingResult(task) + assert(filter.filterTaskGettingResult(taskGettingResultEvent) === Some(true)) + + val taskEndEvent = SparkListenerTaskEnd(stage1.stageId, 0, "taskType", + Success, task, new ExecutorMetrics, null) + assert(filter.filterTaskEnd(taskEndEvent) === Some(true)) + } + } +} From 0729a41489967f404e9e7577bfb0652e15fe6e58 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 13 Nov 2019 19:25:22 +0900 Subject: [PATCH 06/28] Address some more tests - leave new TODO/FIXME for tests --- .../history/EventLogFileCompactor.scala | 41 ++--- .../history/EventLogFileCompactorSuite.scala | 158 ++++++++++++++++++ .../FilteredEventLogFileRewriterSuite.scala | 134 +++++++++++++++ .../history/FsHistoryProviderSuite.scala | 2 + .../status/ListenerEventsTestHelper.scala | 1 + 5 files changed, 309 insertions(+), 27 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/history/FilteredEventLogFileRewriterSuite.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index cdf38fd5669aa..d4a81ed6ecf17 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -35,7 +35,6 @@ import org.apache.spark.internal.config.EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN import org.apache.spark.scheduler._ import org.apache.spark.util.{JsonProtocol, Utils} -// FIXME: UTs class EventLogFileCompactor( sparkConf: SparkConf, hadoopConf: Configuration, @@ -60,11 +59,10 @@ class EventLogFileCompactor( if (filesToCompact.isEmpty) { filesToRetain } else { - // first pass val bus = new ReplayListenerBus() val builders = ServiceLoader.load(classOf[EventFilterBuilder], - Utils.getContextOrSparkClassLoader).asScala + Utils.getContextOrSparkClassLoader).asScala.toSeq builders.foreach(bus.addListener) filesToCompact.foreach { log => @@ -73,17 +71,14 @@ class EventLogFileCompactor( } } - // second pass - val rewriter = new FilteredEventLogFileRewriter(sparkConf, hadoopConf, - filesToCompact, fs, builders.map(_.createFilter()).toSeq) - rewriter.start() - rewriter.rewrite() - rewriter.stop() + val rewriter = new FilteredEventLogFileRewriter(sparkConf, hadoopConf, fs, + builders.map(_.createFilter())) + val compactedPath = rewriter.rewrite(filesToCompact) // cleanup files which are replaced with new compacted file. cleanupCompactedFiles(filesToCompact) - fs.getFileStatus(new Path(rewriter.logPath)) :: filesToRetain.toList + fs.getFileStatus(new Path(compactedPath)) :: filesToRetain.toList } } @@ -112,36 +107,28 @@ class EventLogFileCompactor( } } -// FIXME: UTs class FilteredEventLogFileRewriter( sparkConf: SparkConf, hadoopConf: Configuration, - eventLogFiles: Seq[FileStatus], fs: FileSystem, filters: Seq[EventFilter]) extends Logging { - require(eventLogFiles.nonEmpty) + def rewrite(eventLogFiles: Seq[FileStatus]): String = { + require(eventLogFiles.nonEmpty) - private val targetEventLogFilePath = eventLogFiles.last.getPath - private val logWriter: CompactedEventLogFileWriter = new CompactedEventLogFileWriter( - targetEventLogFilePath, "dummy", None, targetEventLogFilePath.getParent.toUri, - sparkConf, hadoopConf) + val targetEventLogFilePath = eventLogFiles.last.getPath + val logWriter: CompactedEventLogFileWriter = new CompactedEventLogFileWriter( + targetEventLogFilePath, "dummy", None, targetEventLogFilePath.getParent.toUri, + sparkConf, hadoopConf) - def logPath: String = logWriter.logPath - - def start(): Unit = { logWriter.start() - } - - def stop(): Unit = { + eventLogFiles.foreach { file => rewriteFile(logWriter, file) } logWriter.stop() - } - def rewrite(): Unit = { - eventLogFiles.foreach(rewriteFile) + logWriter.logPath } - private def rewriteFile(fileStatus: FileStatus): Unit = { + private def rewriteFile(logWriter: CompactedEventLogFileWriter, fileStatus: FileStatus): Unit = { Utils.tryWithResource(EventLogFileReader.openEventLog(fileStatus.getPath, fs)) { in => val lines = Source.fromInputStream(in)(Codec.UTF8).getLines() diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala new file mode 100644 index 0000000000000..1ecb9fa10f853 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala @@ -0,0 +1,158 @@ +/* + * 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 + +import org.apache.hadoop.fs.{FileStatus, Path} +import org.json4s.jackson.JsonMethods.{compact, render} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.config.EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN +import org.apache.spark.scheduler._ +import org.apache.spark.util.JsonProtocol + +class EventLogFileCompactorSuite extends SparkFunSuite { + private val sparkConf = testSparkConf() + private val hadoopConf = SparkHadoopUtil.newConfiguration(sparkConf) + + test("No event log files") { + withTempDir { dir => + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + assert(Seq.empty[FileStatus] === compactor.compact(Seq.empty)) + } + } + + test("No compact file, less origin files available than max files to retain") { + withTempDir { dir => + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + val logPath1 = writeDummyEventLogFile(dir, 1) + val logPath2 = writeDummyEventLogFile(dir, 2) + + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + val fileStatuses = Seq(logPath1, logPath2).map { p => fs.getFileStatus(new Path(p)) } + val filesToRead = compactor.compact(fileStatuses) + + assert(filesToRead.map(_.getPath) === fileStatuses.map(_.getPath)) + } + } + + test("No compact file, more origin files available than max files to retain") { + withTempDir { dir => + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + val logPaths = (1 to 5).map { idx => writeDummyEventLogFile(dir, idx) } + + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + val fileStatuses = logPaths.map { p => fs.getFileStatus(new Path(p)) } + val filesToRead = compactor.compact(fileStatuses) + + // 3 (max file to retain) + 1 (compacted file) + assert(filesToRead.length === 4) + val originalFilesToRead = filesToRead.takeRight(3) + val originFileToCompact = fileStatuses.takeRight(4).head.getPath + val compactFilePath = filesToRead.head.getPath + + assert(compactFilePath.getName === originFileToCompact.getName + EventLogFileWriter.COMPACTED) + assert(originalFilesToRead.map(_.getPath) === fileStatuses.takeRight(3).map(_.getPath)) + + // compacted files will be removed + fileStatuses.take(2).foreach { status => assert(!fs.exists(status.getPath)) } + filesToRead.foreach { status => assert(fs.exists(status.getPath)) } + } + } + + test("compact file exists, less origin files available than max files to retain") { + withTempDir { dir => + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + val logPaths = (1 to 5).map { idx => writeDummyEventLogFile(dir, idx) } + + val fileToCompact = logPaths(2) + val compactedPath = new Path(fileToCompact + EventLogFileWriter.COMPACTED) + assert(fs.rename(new Path(fileToCompact), compactedPath)) + + val newLogPaths = logPaths.take(2) ++ Seq(compactedPath.toString) ++ + logPaths.takeRight(2) + + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + val fileStatuses = newLogPaths.map { p => fs.getFileStatus(new Path(p)) } + val filesToRead = compactor.compact(fileStatuses) + + // filesToRead will start from rightmost compact file, but no new compact file + assert(filesToRead.map(_.getPath) === fileStatuses.takeRight(3).map(_.getPath)) + } + } + + test("compact file exists, more origin files available than max files to retain") { + withTempDir { dir => + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + val logPaths = (1 to 10).map { idx => writeDummyEventLogFile(dir, idx) } + + val fileToCompact = logPaths(2) + val compactedPath = new Path(fileToCompact + EventLogFileWriter.COMPACTED) + assert(fs.rename(new Path(fileToCompact), compactedPath)) + + val newLogPaths = logPaths.take(2) ++ Seq(compactedPath.toString) ++ + logPaths.takeRight(7) + + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + val fileStatuses = newLogPaths.map { p => fs.getFileStatus(new Path(p)) } + val filesToRead = compactor.compact(fileStatuses) + + // 3 (max file to retain) + 1 (compacted file) + assert(filesToRead.length === 4) + val originalFilesToRead = filesToRead.takeRight(3) + val originFileToCompact = fileStatuses.takeRight(4).head.getPath + val compactFilePath = filesToRead.head.getPath + + assert(compactFilePath.getName === originFileToCompact.getName + EventLogFileWriter.COMPACTED) + assert(originalFilesToRead.map(_.getPath) === fileStatuses.takeRight(3).map(_.getPath)) + + // compacted files will be removed - we don't check files "before" the rightmost compact file + fileStatuses.drop(2).dropRight(3).foreach { status => assert(!fs.exists(status.getPath)) } + filesToRead.foreach { status => assert(fs.exists(status.getPath)) } + } + } + + // TODO: Would we want to verify the result of compact file here? Because we already have it in + // FilteredEventLogFileRewriterSuite, and adding tests everywhere seem to be redundant. + + private def writeDummyEventLogFile(dir: File, idx: Int): String = { + // to simplify the code, we don't concern about file name being matched with the naming rule + // of event log file + val writer = new SingleEventLogFileWriter(s"app$idx", None, dir.toURI, sparkConf, hadoopConf) + writer.start() + writer.writeEvent(convertEvent( + SparkListenerApplicationStart("app", Some("app"), 0, "user", None)), flushLogger = true) + writer.stop() + writer.logPath + } + + private def convertEvent(event: SparkListenerEvent): String = { + compact(render(JsonProtocol.sparkEventToJson(event))) + } + + private def testSparkConf(): SparkConf = { + new SparkConf().set(EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN, 3) + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FilteredEventLogFileRewriterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FilteredEventLogFileRewriterSuite.scala new file mode 100644 index 0000000000000..a554fbcc2031a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/FilteredEventLogFileRewriterSuite.scala @@ -0,0 +1,134 @@ +/* + * 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 scala.collection.mutable +import scala.io.{Codec, Source} + +import org.apache.hadoop.fs.Path +import org.json4s.jackson.JsonMethods.{compact, render} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.scheduler._ +import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.{JsonProtocol, Utils} + +class FilteredEventLogFileRewriterSuite extends SparkFunSuite { + test("rewrite files with test filters") { + def writeEventToWriter(writer: EventLogFileWriter, event: SparkListenerEvent): String = { + val line = convertEvent(event) + writer.writeEvent(line, flushLogger = true) + line + } + + withTempDir { tempDir => + val sparkConf = new SparkConf + val hadoopConf = SparkHadoopUtil.newConfiguration(sparkConf) + val fs = new Path(tempDir.getAbsolutePath).getFileSystem(hadoopConf) + + val writer = new SingleEventLogFileWriter("app", None, tempDir.toURI, sparkConf, hadoopConf) + writer.start() + + val expectedLines = new mutable.ArrayBuffer[String] + + // filterApplicationEnd: Some(true) & Some(true) => filter in + expectedLines += writeEventToWriter(writer, SparkListenerApplicationEnd(0)) + + // filterBlockManagerAdded: Some(true) & Some(false) => filter out + writeEventToWriter(writer, SparkListenerBlockManagerAdded(0, BlockManagerId("1", "host1", 1), + 10)) + + // filterApplicationStart: Some(false) & Some(false) => filter out + writeEventToWriter(writer, SparkListenerApplicationStart("app", None, 0, "user", None)) + + // filterNodeBlacklisted: None & Some(true) => filter in + expectedLines += writeEventToWriter(writer, SparkListenerNodeBlacklisted(0, "host1", 1)) + + // filterNodeUnblacklisted: None & Some(false) => filter out + writeEventToWriter(writer, SparkListenerNodeUnblacklisted(0, "host1")) + + // other events: None & None => filter in + expectedLines += writeEventToWriter(writer, SparkListenerUnpersistRDD(0)) + + writer.stop() + + val filters = Seq(new TestEventFilter1, new TestEventFilter2) + + val rewriter = new FilteredEventLogFileRewriter(sparkConf, hadoopConf, fs, filters) + val logPath = new Path(writer.logPath) + val newPath = rewriter.rewrite(Seq(fs.getFileStatus(logPath))) + assert(new Path(newPath).getName === logPath.getName + EventLogFileWriter.COMPACTED) + + Utils.tryWithResource(EventLogFileReader.openEventLog(new Path(newPath), fs)) { is => + val lines = Source.fromInputStream(is)(Codec.UTF8).getLines() + var linesLength = 0 + lines.foreach { line => + linesLength += 1 + assert(expectedLines.contains(line)) + } + assert(linesLength === expectedLines.length, "The number of lines for rewritten file " + + s"is not expected: expected ${expectedLines.length} / actual $linesLength") + } + } + } + + private def convertEvent(event: SparkListenerEvent): String = { + compact(render(JsonProtocol.sparkEventToJson(event))) + } +} + +class TestEventFilter1 extends EventFilter { + override def filterApplicationEnd(event: SparkListenerApplicationEnd): Option[Boolean] = { + Some(true) + } + + override def filterBlockManagerAdded(event: SparkListenerBlockManagerAdded): Option[Boolean] = { + Some(true) + } + + override def filterApplicationStart(event: SparkListenerApplicationStart): Option[Boolean] = { + Some(false) + } +} + +class TestEventFilter2 extends EventFilter { + override def filterApplicationEnd(event: SparkListenerApplicationEnd): Option[Boolean] = { + Some(true) + } + + override def filterEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Option[Boolean] = { + Some(true) + } + + override def filterBlockManagerAdded(event: SparkListenerBlockManagerAdded): Option[Boolean] = { + Some(false) + } + + override def filterApplicationStart(event: SparkListenerApplicationStart): Option[Boolean] = { + Some(false) + } + + override def filterNodeBlacklisted(event: SparkListenerNodeBlacklisted): Option[Boolean] = { + Some(true) + } + + override def filterNodeUnblacklisted(event: SparkListenerNodeUnblacklisted): Option[Boolean] = { + Some(false) + } +} 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 ed195dd44e917..6259276b1a9d8 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 @@ -1333,6 +1333,8 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { } } + // FIXME: add end-to-end test for compaction, or mock to verify whether compact is being called + /** * Asks the provider to check for logs and calls a function to perform checks on the updated * app list. Example: diff --git a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala index cf96c3a52d6f7..f9496ecc4740a 100644 --- a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala +++ b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala @@ -20,6 +20,7 @@ package org.apache.spark.status import java.util.Properties import scala.collection.immutable.Map + import org.apache.spark.{AccumulatorSuite, SparkContext} import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.scheduler.{SparkListenerExecutorAdded, SparkListenerExecutorMetricsUpdate, SparkListenerExecutorRemoved, SparkListenerStageCompleted, SparkListenerStageSubmitted, SparkListenerTaskStart, StageInfo, TaskInfo, TaskLocality} From 316bc172b8678a1e05c974db8be904ab343050b2 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 13 Nov 2019 19:38:31 +0900 Subject: [PATCH 07/28] Address review comments --- .../apache/spark/deploy/history/EventFilter.scala | 4 ++-- .../spark/deploy/history/EventLogFileCompactor.scala | 12 ++++++++---- .../spark/deploy/history/EventLogFileReaders.scala | 2 +- .../spark/deploy/history/EventLogFileWriters.scala | 8 ++++++-- 4 files changed, 17 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala index da73908bb8ad1..6bb9022241f31 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala @@ -19,11 +19,11 @@ package org.apache.spark.deploy.history import org.apache.spark.scheduler._ -trait EventFilterBuilder extends SparkListenerInterface { +private[spark] trait EventFilterBuilder extends SparkListenerInterface { def createFilter(): EventFilter } -trait EventFilter { +private[spark] trait EventFilter { def filterStageCompleted(event: SparkListenerStageCompleted): Option[Boolean] = None def filterStageSubmitted(event: SparkListenerStageSubmitted): Option[Boolean] = None diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index d4a81ed6ecf17..baa40b9044b01 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -46,8 +46,8 @@ class EventLogFileCompactor( // sequentially if the last event log file is already a compacted file, everything // will be skipped def compact(eventLogFiles: Seq[FileStatus]): Seq[FileStatus] = { - if (eventLogFiles.isEmpty) { - return Seq.empty[FileStatus] + if (eventLogFiles.length <= maxFilesToRetain) { + return eventLogFiles } // skip everything if the last file is already a compacted file @@ -84,10 +84,14 @@ class EventLogFileCompactor( private def cleanupCompactedFiles(files: Seq[FileStatus]): Unit = { files.foreach { file => + var deleted = false try { - fs.delete(file.getPath, true) + deleted = fs.delete(file.getPath, true) } catch { - case _: IOException => logWarning(s"Failed to remove ${file.getPath} / skip removing.") + case _: IOException => + } + if (!deleted) { + logWarning(s"Failed to remove ${file.getPath} / skip removing.") } } } 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 597fd3f2671f9..2ec41bfa038ef 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 @@ -220,7 +220,7 @@ class RollingEventLogFilesFileReader( private lazy val eventLogFiles: Seq[FileStatus] = { val eventLogFiles = files.filter(isEventLogFile).sortBy { status => val filePath = status.getPath - var idx = getIndex(status.getPath.getName) + 0.0d + var idx = getIndex(filePath.getName) + 0.0d // trick to place compacted file later than normal file if index is same. if (EventLogFileWriter.isCompacted(filePath)) { idx += 0.1 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 d5f413e681a2c..8b497f4cbea93 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 @@ -402,8 +402,12 @@ object RollingEventLogFilesWriter { status.isDirectory && status.getPath.getName.startsWith(EVENT_LOG_DIR_NAME_PREFIX) } + def isEventLogFile(fileName: String): Boolean = { + fileName.startsWith(EVENT_LOG_FILE_NAME_PREFIX) + } + def isEventLogFile(status: FileStatus): Boolean = { - status.isFile && status.getPath.getName.startsWith(EVENT_LOG_FILE_NAME_PREFIX) + status.isFile && isEventLogFile(status.getPath.getName) } def isAppStatusFile(status: FileStatus): Boolean = { @@ -411,7 +415,7 @@ object RollingEventLogFilesWriter { } def getIndex(eventLogFileName: String): Long = { - require(eventLogFileName.startsWith(EVENT_LOG_FILE_NAME_PREFIX), "Not an event log file!") + require(isEventLogFile(eventLogFileName), "Not an event log file!") val index = eventLogFileName.stripPrefix(EVENT_LOG_FILE_NAME_PREFIX).split("_")(0) index.toLong } From b8a4bc140b1c20af2f0e4299cc363e51251f9d3f Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 14 Nov 2019 07:31:10 +0900 Subject: [PATCH 08/28] Review comment & fix the test --- .../org/apache/spark/deploy/history/EventLogFileCompactor.scala | 2 ++ .../org/apache/spark/deploy/history/EventLogFileReaders.scala | 2 +- .../org/apache/spark/status/ListenerEventsTestHelper.scala | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index baa40b9044b01..4fb1b5fc8b0bf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -176,6 +176,8 @@ class FilteredEventLogFileRewriter( } private def applyFilter(filter: EventFilter, event: SparkListenerEvent): Option[Boolean] = { + // This pattern match should have same list of event types, but it would be safe even if + // it's out of sync, once filter doesn't mark events to filter out for unknown event types. event match { case event: SparkListenerStageSubmitted => filter.filterStageSubmitted(event) case event: SparkListenerStageCompleted => filter.filterStageCompleted(event) 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 2ec41bfa038ef..f13abaab14329 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 @@ -220,7 +220,7 @@ class RollingEventLogFilesFileReader( private lazy val eventLogFiles: Seq[FileStatus] = { val eventLogFiles = files.filter(isEventLogFile).sortBy { status => val filePath = status.getPath - var idx = getIndex(filePath.getName) + 0.0d + var idx = getIndex(filePath.getName).toDouble // trick to place compacted file later than normal file if index is same. if (EventLogFileWriter.isCompacted(filePath)) { idx += 0.1 diff --git a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala index f9496ecc4740a..12e456f97ab20 100644 --- a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala +++ b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala @@ -116,7 +116,7 @@ object ListenerEventsTestHelper { } def createExecutorRemovedEvent(executorId: Int): SparkListenerExecutorRemoved = { - createExecutorRemovedEvent(executorId.toString, 0) + createExecutorRemovedEvent(executorId.toString, 10L) } /** Create an executor added event for the specified executor Id. */ From 1106bcbb47afbf1ebaffc4d2e38e21f4746c7f97 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 14 Nov 2019 10:52:52 +0900 Subject: [PATCH 09/28] Add more tests - TODO for tests addressed --- .../deploy/history/FsHistoryProvider.scala | 4 +- .../history/EventLogFileCompactorSuite.scala | 55 ++++++++++++----- .../deploy/history/EventLogTestHelper.scala | 24 ++++++++ .../history/FsHistoryProviderSuite.scala | 61 ++++++++++++++++++- 4 files changed, 124 insertions(+), 20 deletions(-) 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 432fc92671806..b5b0816fc39d3 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 @@ -944,9 +944,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } /** - * Rebuilds the application state store from its event log. + * Rebuilds the application state store from its event log. Exposed for testing. */ - private def rebuildAppStore( + private[spark] def rebuildAppStore( store: KVStore, reader: EventLogFileReader, lastUpdated: Long): Unit = { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala index 1ecb9fa10f853..2cdca9db84c54 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala @@ -19,14 +19,17 @@ package org.apache.spark.deploy.history import java.io.File +import scala.io.{Codec, Source} + import org.apache.hadoop.fs.{FileStatus, Path} -import org.json4s.jackson.JsonMethods.{compact, render} +import org.json4s.jackson.JsonMethods.parse import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.config.EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN import org.apache.spark.scheduler._ -import org.apache.spark.util.JsonProtocol +import org.apache.spark.util.{JsonProtocol, Utils} + class EventLogFileCompactorSuite extends SparkFunSuite { private val sparkConf = testSparkConf() @@ -134,22 +137,44 @@ class EventLogFileCompactorSuite extends SparkFunSuite { } } - // TODO: Would we want to verify the result of compact file here? Because we already have it in - // FilteredEventLogFileRewriterSuite, and adding tests everywhere seem to be redundant. + test("events for finished job are filtered out in new compact file") { + withTempDir { dir => + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + // 1, 2 will be compacted into one file, 3~5 are dummies to ensure max files to retain + val logPath1 = EventLogTestHelper.writeEventLogFile(sparkConf, hadoopConf, dir, 1, Seq( + SparkListenerUnpersistRDD(0), SparkListenerJobStart(1, 0, Seq.empty))) + val logPath2 = EventLogTestHelper.writeEventLogFile(sparkConf, hadoopConf, dir, 2, Seq( + SparkListenerJobEnd(1, 1, JobSucceeded), SparkListenerUnpersistRDD(1))) + val logPaths = Seq(logPath1, logPath2) ++ (3 to 5).map { idx => + writeDummyEventLogFile(dir, idx) + } - private def writeDummyEventLogFile(dir: File, idx: Int): String = { - // to simplify the code, we don't concern about file name being matched with the naming rule - // of event log file - val writer = new SingleEventLogFileWriter(s"app$idx", None, dir.toURI, sparkConf, hadoopConf) - writer.start() - writer.writeEvent(convertEvent( - SparkListenerApplicationStart("app", Some("app"), 0, "user", None)), flushLogger = true) - writer.stop() - writer.logPath + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + val fileStatuses = logPaths.map { p => fs.getFileStatus(new Path(p)) } + val filesToRead = compactor.compact(fileStatuses) + + // 3 (max file to retain) + 1 (compacted file) + assert(filesToRead.length === 4) + val compactFilePath = filesToRead.head.getPath + + Utils.tryWithResource(EventLogFileReader.openEventLog(compactFilePath, fs)) { is => + val lines = Source.fromInputStream(is)(Codec.UTF8).getLines() + var linesLength = 0 + lines.foreach { line => + linesLength += 1 + val event = JsonProtocol.sparkEventFromJson(parse(line)) + assert(!event.isInstanceOf[SparkListenerJobStart] && + !event.isInstanceOf[SparkListenerJobEnd]) + } + assert(linesLength === 2, "Compacted file should have only two events being filtered in") + } + } } - private def convertEvent(event: SparkListenerEvent): String = { - compact(render(JsonProtocol.sparkEventToJson(event))) + private def writeDummyEventLogFile(dir: File, idx: Int): String = { + EventLogTestHelper.writeEventLogFile(sparkConf, hadoopConf, dir, idx, + Seq(SparkListenerApplicationStart("app", Some("app"), 0, "user", None))) } private def testSparkConf(): SparkConf = { 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 55eddce3968c2..80cc1fc842ed1 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,12 +17,17 @@ package org.apache.spark.deploy.history +import java.io.File import java.nio.charset.StandardCharsets +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.json4s.jackson.JsonMethods.{compact, render} import org.apache.spark.SparkConf import org.apache.spark.internal.config._ +import org.apache.spark.scheduler.SparkListenerEvent +import org.apache.spark.util.JsonProtocol object EventLogTestHelper { def getUniqueApplicationId: String = "test-" + System.currentTimeMillis @@ -56,4 +61,23 @@ object EventLogTestHelper { eventStr } } + + def writeEventLogFile( + sparkConf: SparkConf, + hadoopConf: Configuration, + dir: File, + idx: Int, + events: Seq[SparkListenerEvent]): String = { + // to simplify the code, we don't concern about file name being matched with the naming rule + // of event log file + val writer = new SingleEventLogFileWriter(s"app$idx", None, dir.toURI, sparkConf, hadoopConf) + writer.start() + events.foreach { event => writer.writeEvent(convertEvent(event), flushLogger = true) } + writer.stop() + writer.logPath + } + + def convertEvent(event: SparkListenerEvent): String = { + compact(render(JsonProtocol.sparkEventToJson(event))) + } } 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 6259276b1a9d8..4fd3f50338ff8 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 @@ -37,9 +37,10 @@ import org.mockito.Mockito.{doThrow, mock, spy, verify, when} import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ -import org.apache.spark.{SecurityManager, SPARK_VERSION, SparkConf, SparkFunSuite} +import org.apache.spark.{JobExecutionStatus, SecurityManager, SPARK_VERSION, SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.DRIVER_LOG_DFS_DIR +import org.apache.spark.internal.config.{DRIVER_LOG_DFS_DIR, EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN} import org.apache.spark.internal.config.History._ import org.apache.spark.internal.config.UI.{ADMIN_ACLS, ADMIN_ACLS_GROUPS, USER_GROUPS_MAPPING} import org.apache.spark.io._ @@ -50,6 +51,7 @@ 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.kvstore.InMemoryStore import org.apache.spark.util.logging.DriverLogger class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { @@ -1333,7 +1335,60 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { } } - // FIXME: add end-to-end test for compaction, or mock to verify whether compact is being called + test("compact event log files when replaying to rebuild app") { + withTempDir { dir => + val conf = createTestConf() + conf.set(EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN, 1) + val hadoopConf = SparkHadoopUtil.newConfiguration(conf) + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + // 1, 2 will be compacted into one file, 3 is the dummy file to ensure max files to retain + val logPath1 = EventLogTestHelper.writeEventLogFile(conf, hadoopConf, dir, 1, Seq( + SparkListenerApplicationStart("app", Some("app"), 0, "user", None), + SparkListenerJobStart(1, 0, Seq.empty))) + val logPath2 = EventLogTestHelper.writeEventLogFile(conf, hadoopConf, dir, 2, Seq( + SparkListenerUnpersistRDD(1), SparkListenerJobEnd(1, 1, JobSucceeded))) + val logPath3 = EventLogTestHelper.writeEventLogFile(conf, hadoopConf, dir, 3, Seq( + SparkListenerExecutorAdded(3, "exec1", new ExecutorInfo("host1", 1, Map.empty)), + SparkListenerJobStart(2, 4, Seq.empty), + SparkListenerJobEnd(2, 5, JobSucceeded))) + val logPaths = Seq(logPath1, logPath2, logPath3) + + val store = new InMemoryStore + val appStore = new AppStatusStore(store) + val reader = mock(classOf[EventLogFileReader]) + when(reader.listEventLogFiles).thenReturn(logPaths.map { p => fs.getFileStatus(new Path(p)) }) + + val provider = new FsHistoryProvider(conf) + provider.rebuildAppStore(store, reader, 0) + + // files being compacted are deleted + assert(!fs.exists(new Path(logPath1))) + assert(!fs.exists(new Path(logPath2))) + + // new compact file should be available + assert(fs.exists(new Path(logPath2 + EventLogFileWriter.COMPACTED))) + + // retained file is not touched + assert(fs.exists(new Path(logPath3))) + + // replayed store doesn't have any job, as events for job are removed while compacting + intercept[NoSuchElementException] { + appStore.job(1) + } + + // but other events should be available even they were in original files to compact + val appInfo = appStore.applicationInfo() + assert(appInfo.id === "app") + assert(appInfo.name === "app") + + // all events in retained file should be available, even they're related to finished jobs + val exec1 = appStore.executorSummary("exec1") + assert(exec1.hostPort === "host1") + val job2 = appStore.job(2) + assert(job2.status === JobExecutionStatus.SUCCEEDED) + } + } /** * Asks the provider to check for logs and calls a function to perform checks on the updated From 5bd4274e4e28a9a08103b50ced13647059218ea7 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 14 Nov 2019 13:35:05 +0900 Subject: [PATCH 10/28] Fix UT (silly mistake) --- .../deploy/history/EventLogFileCompactorSuite.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala index 2cdca9db84c54..33b4404d1a18f 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala @@ -20,14 +20,13 @@ package org.apache.spark.deploy.history import java.io.File import scala.io.{Codec, Source} - import org.apache.hadoop.fs.{FileStatus, Path} import org.json4s.jackson.JsonMethods.parse - import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.config.EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.{JsonProtocol, Utils} @@ -143,9 +142,11 @@ class EventLogFileCompactorSuite extends SparkFunSuite { // 1, 2 will be compacted into one file, 3~5 are dummies to ensure max files to retain val logPath1 = EventLogTestHelper.writeEventLogFile(sparkConf, hadoopConf, dir, 1, Seq( - SparkListenerUnpersistRDD(0), SparkListenerJobStart(1, 0, Seq.empty))) + SparkListenerExecutorAdded(0, "exec1", new ExecutorInfo("host1", 1, Map.empty)), + SparkListenerJobStart(1, 0, Seq.empty))) val logPath2 = EventLogTestHelper.writeEventLogFile(sparkConf, hadoopConf, dir, 2, Seq( - SparkListenerJobEnd(1, 1, JobSucceeded), SparkListenerUnpersistRDD(1))) + SparkListenerJobEnd(1, 1, JobSucceeded), + SparkListenerExecutorAdded(2, "exec2", new ExecutorInfo("host2", 1, Map.empty)))) val logPaths = Seq(logPath1, logPath2) ++ (3 to 5).map { idx => writeDummyEventLogFile(dir, idx) } From 45e828ecdd2992b98c0c3344107d362aec662415 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 14 Nov 2019 13:35:23 +0900 Subject: [PATCH 11/28] Address scaladoc --- .../history/BasicEventFilterBuilder.scala | 10 ++++++ .../spark/deploy/history/EventFilter.scala | 16 +++++++++ .../history/EventLogFileCompactor.scala | 35 ++++++++++++++++--- .../history/EventLogFileCompactorSuite.scala | 3 +- .../history/SQLEventFilterBuilder.scala | 18 ++++++++++ 5 files changed, 76 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala index 55c50b92cbd7d..5f9617e30bf9d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala @@ -22,6 +22,11 @@ import scala.collection.mutable import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ +/** + * This class tracks both live jobs and live executors, and pass the list to the + * [[BasicEventFilter]] to help BasicEventFilter to filter out finished jobs + * (+ stages/tasks/RDDs) and dead executors. + */ private[spark] class BasicEventFilterBuilder extends SparkListener with EventFilterBuilder { private val _liveJobToStages = new mutable.HashMap[Int, Seq[Int]] private val _stageToTasks = new mutable.HashMap[Int, mutable.Set[Long]] @@ -66,6 +71,11 @@ private[spark] class BasicEventFilterBuilder extends SparkListener with EventFil override def createFilter(): EventFilter = BasicEventFilter(this) } +/** + * This class filters out events which are related to the finished jobs or dead executors, + * based on the given information. The events which are not related to the job and executor + * will be considered as "Don't mind". + */ private[spark] class BasicEventFilter( liveJobToStages: Map[Int, Seq[Int]], stageToTasks: Map[Int, Set[Long]], diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala index 6bb9022241f31..5fbd9951af559 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala @@ -19,10 +19,26 @@ package org.apache.spark.deploy.history import org.apache.spark.scheduler._ +/** + * EventFilterBuilder provides the interface to gather the information from events being received + * by [[SparkListenerInterface]], and create a new [[EventFilter]] instance which leverages + * information gathered to decide whether the event should be filtered or not. + */ private[spark] trait EventFilterBuilder extends SparkListenerInterface { def createFilter(): EventFilter } +/** + * [[EventFilter]] decides whether the given event should be filtered in, or filtered out when + * compacting event log files. + * + * The meaning of return values of each filterXXX method are following: + * - Some(true): Filter in this event. + * - Some(false): Filter out this event. + * - None: Don't mind about this event. No problem even other filters decide to filter out. + * + * Please refer [[FilteredEventLogFileRewriter]] for more details on how the filter will be used. + */ private[spark] trait EventFilter { def filterStageCompleted(event: SparkListenerStageCompleted): Option[Boolean] = None diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index 4fb1b5fc8b0bf..fdf945181b5fd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -35,6 +35,25 @@ import org.apache.spark.internal.config.EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN import org.apache.spark.scheduler._ import org.apache.spark.util.{JsonProtocol, Utils} +/** + * This class compacts the old event log files into one compact file, via two phases reading: + * + * 1) Initialize available [[EventFilterBuilder]] instances, and replay the old event log files with + * builders, so that these builders can gather the information to create [[EventFilter]] instances. + * 2) Initialize [[EventFilter]] instances from [[EventFilterBuilder]] instances, and replay the + * old event log files with filters. Rewrite the content to the compact file if the filters decide + * to filter in. + * + * This class assumes caller will provide the sorted list of files which are sorted by the index of + * event log file - caller should keep in mind that this class doesn't care about the semantic of + * ordering. + * + * When compacting the files, the range of compaction for given file list is determined as: + * (rightmost compact file ~ the file where there're `maxFilesToRetain` files on the right side) + * + * If there's no compact file in the list, it starts from the first file. If there're not enough + * files after rightmost compact file, compaction will be skipped. + */ class EventLogFileCompactor( sparkConf: SparkConf, hadoopConf: Configuration, @@ -42,15 +61,11 @@ class EventLogFileCompactor( private val maxFilesToRetain: Int = sparkConf.get(EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN) - // FIXME: javadoc - caller should provide event log files (either compacted or original) - // sequentially if the last event log file is already a compacted file, everything - // will be skipped def compact(eventLogFiles: Seq[FileStatus]): Seq[FileStatus] = { if (eventLogFiles.length <= maxFilesToRetain) { return eventLogFiles } - // skip everything if the last file is already a compacted file if (EventLogFileWriter.isCompacted(eventLogFiles.last.getPath)) { return Seq(eventLogFiles.last) } @@ -75,7 +90,6 @@ class EventLogFileCompactor( builders.map(_.createFilter())) val compactedPath = rewriter.rewrite(filesToCompact) - // cleanup files which are replaced with new compacted file. cleanupCompactedFiles(filesToCompact) fs.getFileStatus(new Path(compactedPath)) :: filesToRetain.toList @@ -111,6 +125,12 @@ class EventLogFileCompactor( } } +/** + * This class rewrites the event log files into one compact file: the compact file will only + * contain the events which pass the filters. Events will be filtered out only when all filters + * decide to filter out the event or don't mind about the event. Otherwise, the original line for + * the event is written to the compact file as it is. + */ class FilteredEventLogFileRewriter( sparkConf: SparkConf, hadoopConf: Configuration, @@ -212,6 +232,11 @@ class FilteredEventLogFileRewriter( } } +/** + * This class helps to write compact file; to avoid reimplement everything, it extends + * [[SingleEventLogFileWriter]], but only `originalFilePath` is used to determine the + * path of compact file. + */ class CompactedEventLogFileWriter( originalFilePath: Path, appId: String, diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala index 33b4404d1a18f..9a3f3784b774b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala @@ -20,8 +20,10 @@ package org.apache.spark.deploy.history import java.io.File import scala.io.{Codec, Source} + import org.apache.hadoop.fs.{FileStatus, Path} import org.json4s.jackson.JsonMethods.parse + import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.config.EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN @@ -29,7 +31,6 @@ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.{JsonProtocol, Utils} - class EventLogFileCompactorSuite extends SparkFunSuite { private val sparkConf = testSparkConf() private val hadoopConf = SparkHadoopUtil.newConfiguration(sparkConf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala index 07795313002e1..268402177413a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala @@ -26,6 +26,12 @@ import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.ui._ import org.apache.spark.sql.streaming.StreamingQueryListener +/** + * This class tracks live SQL executions, and pass the list to the [[SQLLiveEntitiesEventFilter]] + * to help SQLLiveEntitiesEventFilter to filter out finished SQL executions as well as relevant + * jobs (+ stages/tasks/RDDs). Unlike BasicEventFilterBuilder, it doesn't concern about the status + * of individual job - it only concerns whether SQL execution is finished or not. + */ private[spark] class SQLEventFilterBuilder extends SparkListener with EventFilterBuilder { private val _liveExecutionToJobs = new mutable.HashMap[Long, mutable.Set[Int]] private val _jobToStages = new mutable.HashMap[Int, Seq[Int]] @@ -98,6 +104,18 @@ private[spark] class SQLEventFilterBuilder extends SparkListener with EventFilte } } +/** + * This class filters out events which are related to the finished SQL executions based on the + * given information. + * + * Note that filterXXX methods will return None instead of Some(false) if the event is related to + * job but not coupled with live SQL executions, because the instance has the information about + * jobs for live SQL executions which should be filtered in, but don't know whether the job is + * related to the finished SQL executions, or job is NOT related to the SQL executions. For this + * case, it just gives up the decision and let other filters decide it. + * + * The events which are not related to the SQL execution will be considered as "Don't mind". + */ private[spark] class SQLLiveEntitiesEventFilter( liveExecutionToJobs: Map[Long, Set[Int]], jobToStages: Map[Int, Seq[Int]], From f64388a37fb61bf042b9df002cf48fd51b07c351 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 14 Nov 2019 13:53:18 +0900 Subject: [PATCH 12/28] Address documentation --- docs/configuration.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index 97ea1fb4ba041..e703cf2b6d180 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1023,6 +1023,17 @@ Apart from these, the following properties are also available, and may be useful The max size of event log file before it's rolled over. + + spark.eventLog.rolling.maxFilesToRetain + Int.MaxValue + + The maximum number of event log files which will be retained as non-compacted. + By default, all event log files will be retained. Please set the configuration and + spark.eventLog.rolling.maxFileSize accordingly if you want to control + the overall size of event log files. The event log files older than these retained + files will be compacted into single file and deleted afterwards. + + spark.ui.dagGraph.retainedRootRDDs Int.MaxValue From 4559c0dae8c7b9e0e924c55e796506eb4f8ee415 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 14 Nov 2019 17:53:31 +0900 Subject: [PATCH 13/28] Clean up codebase a bit --- .../history/BasicEventFilterBuilder.scala | 58 +++++++--- .../history/EventLogFileCompactor.scala | 7 +- .../deploy/history/EventLogFileReaders.scala | 15 ++- .../history/BasicEventFilterSuite.scala | 101 ++++++++++-------- .../history/EventLogFileCompactorSuite.scala | 48 ++++----- .../history/SQLEventFilterBuilder.scala | 56 ++++------ .../history/SQLEventFilterBuilderSuite.scala | 100 ++++++++--------- 7 files changed, 202 insertions(+), 183 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala index 5f9617e30bf9d..f8de6663b2bb4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala @@ -72,15 +72,15 @@ private[spark] class BasicEventFilterBuilder extends SparkListener with EventFil } /** - * This class filters out events which are related to the finished jobs or dead executors, - * based on the given information. The events which are not related to the job and executor - * will be considered as "Don't mind". + * This class provides the functionality to filter out events which are related to the finished + * jobs based on the given information. This class only deals with job related events, and returns + * either Some(true) or Some(false) - successors should override the methods if they don't want to + * return Some(false) for finished jobs and related events. */ -private[spark] class BasicEventFilter( - liveJobToStages: Map[Int, Seq[Int]], +private[spark] abstract class JobEventFilter( + jobToStages: Map[Int, Seq[Int]], stageToTasks: Map[Int, Set[Long]], - stageToRDDs: Map[Int, Seq[Int]], - liveExecutors: Set[String]) extends EventFilter with Logging { + stageToRDDs: Map[Int, Seq[Int]]) extends EventFilter with Logging { private val liveTasks: Set[Long] = stageToTasks.values match { case xs if xs.isEmpty => Set.empty[Long] @@ -93,8 +93,8 @@ private[spark] class BasicEventFilter( } if (log.isDebugEnabled) { - logDebug(s"live jobs : ${liveJobToStages.keySet}") - logDebug(s"stages in jobs : ${liveJobToStages.values.flatten}") + logDebug(s"jobs : ${jobToStages.keySet}") + logDebug(s"stages in jobs : ${jobToStages.values.flatten}") logDebug(s"stages : ${stageToTasks.keySet}") logDebug(s"tasks in stages : ${stageToTasks.values.flatten}") logDebug(s"RDDs in stages : ${stageToRDDs.values.flatten}") @@ -121,20 +121,44 @@ private[spark] class BasicEventFilter( } override def filterJobStart(event: SparkListenerJobStart): Option[Boolean] = { - Some(liveJobToStages.contains(event.jobId)) + Some(jobToStages.contains(event.jobId)) } override def filterJobEnd(event: SparkListenerJobEnd): Option[Boolean] = { - Some(liveJobToStages.contains(event.jobId)) + Some(jobToStages.contains(event.jobId)) } override def filterUnpersistRDD(event: SparkListenerUnpersistRDD): Option[Boolean] = { Some(liveRDDs.contains(event.rddId)) } - override def filterStageExecutorMetrics( - event: SparkListenerStageExecutorMetrics): Option[Boolean] = { - Some(liveExecutors.contains(event.execId)) + override def filterExecutorMetricsUpdate( + event: SparkListenerExecutorMetricsUpdate): Option[Boolean] = { + Some(event.accumUpdates.exists { case (_, stageId, _, _) => + stageToTasks.contains(stageId) + }) + } + + override def filterSpeculativeTaskSubmitted( + event: SparkListenerSpeculativeTaskSubmitted): Option[Boolean] = { + Some(stageToTasks.contains(event.stageId)) + } +} + +/** + * This class filters out events which are related to the finished jobs or dead executors, + * based on the given information. The events which are not related to the job and executor + * will be considered as "Don't mind". + */ +private[spark] class BasicEventFilter( + _liveJobToStages: Map[Int, Seq[Int]], + _stageToTasks: Map[Int, Set[Long]], + _stageToRDDs: Map[Int, Seq[Int]], + liveExecutors: Set[String]) + extends JobEventFilter(_liveJobToStages, _stageToTasks, _stageToRDDs) with Logging { + + if (log.isDebugEnabled) { + logDebug(s"live executors : $liveExecutors") } override def filterExecutorAdded(event: SparkListenerExecutorAdded): Option[Boolean] = { @@ -155,9 +179,9 @@ private[spark] class BasicEventFilter( Some(liveExecutors.contains(event.executorId)) } - override def filterSpeculativeTaskSubmitted( - event: SparkListenerSpeculativeTaskSubmitted): Option[Boolean] = { - Some(stageToTasks.contains(event.stageId)) + override def filterStageExecutorMetrics( + event: SparkListenerStageExecutorMetrics): Option[Boolean] = { + Some(liveExecutors.contains(event.execId)) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index fdf945181b5fd..08139e314ab35 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -112,11 +112,7 @@ class EventLogFileCompactor( private def findFilesToCompact( eventLogFiles: Seq[FileStatus]): (Seq[FileStatus], Seq[FileStatus]) = { - val lastCompactedFileIdx = eventLogFiles.lastIndexWhere { fs => - EventLogFileWriter.isCompacted(fs.getPath) - } - val files = eventLogFiles.drop(lastCompactedFileIdx) - + val files = RollingEventLogFilesFileReader.dropBeforeLastCompactFile(eventLogFiles) if (files.length > maxFilesToRetain) { (files.dropRight(maxFilesToRetain), files.takeRight(maxFilesToRetain)) } else { @@ -161,7 +157,6 @@ class FilteredEventLogFileRewriter( try { val lineEntries = lines.zipWithIndex - while (lineEntries.hasNext) { try { val entry = lineEntries.next() 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 f13abaab14329..bcaa8d0901f8c 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 @@ -228,11 +228,7 @@ class RollingEventLogFilesFileReader( idx } - val lastCompactedFileIdx = eventLogFiles.lastIndexWhere { fs => - EventLogFileWriter.isCompacted(fs.getPath) - } - val filesToRead = eventLogFiles.drop(lastCompactedFileIdx) - + val filesToRead = RollingEventLogFilesFileReader.dropBeforeLastCompactFile(eventLogFiles) val indices = filesToRead.map { file => getIndex(file.getPath.getName) } require((indices.head to indices.last) == indices, "Found missing event log file, expected" + s" indices: ${(indices.head to indices.last)}, actual: ${indices}") @@ -275,3 +271,12 @@ class RollingEventLogFilesFileReader( private def lastEventLogFile: FileStatus = eventLogFiles.last } + +object RollingEventLogFilesFileReader { + def dropBeforeLastCompactFile(eventLogFiles: Seq[FileStatus]): Seq[FileStatus] = { + val lastCompactedFileIdx = eventLogFiles.lastIndexWhere { fs => + EventLogFileWriter.isCompacted(fs.getPath) + } + eventLogFiles.drop(lastCompactedFileIdx) + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala index 8395b2fb180ff..5e9b067a9607b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala @@ -49,33 +49,20 @@ class BasicEventFilterSuite extends SparkFunSuite { val unpersistRDDEventsForJob1 = (1 to 2).map(SparkListenerUnpersistRDD) // job events for finished job should be filtered out - assert(filter.filterJobStart(jobStartEventForJob1) === Some(false)) - assert(filter.filterJobEnd(jobEndEventForJob1) === Some(false)) + assertFilterJobEvents(filter, jobStartEventForJob1, jobEndEventForJob1, Some(false)) // stage events for finished job should be filtered out // NOTE: it doesn't filter out stage events which are also related to the executor - assert(filter.filterStageSubmitted(stageSubmittedEventsForJob1) === Some(false)) - assert(filter.filterStageCompleted(stageCompletedEventsForJob1) === Some(false)) - unpersistRDDEventsForJob1.foreach { event => - assert(filter.filterUnpersistRDD(event) === Some(false)) - } - - val taskSpeculativeTaskSubmittedEvent = SparkListenerSpeculativeTaskSubmitted(stage1.stageId, - stageAttemptId = 1) - assert(filter.filterSpeculativeTaskSubmitted(taskSpeculativeTaskSubmittedEvent) === Some(false)) + assertFilterStageEvents( + filter, + stageSubmittedEventsForJob1, + stageCompletedEventsForJob1, + unpersistRDDEventsForJob1, + SparkListenerSpeculativeTaskSubmitted(stage1.stageId, stageAttemptId = 1), + Some(false)) // task events for finished job should be filtered out - tasksForStage1.foreach { task => - val taskStartEvent = SparkListenerTaskStart(stage1.stageId, 0, task) - assert(filter.filterTaskStart(taskStartEvent) === Some(false)) - - val taskGettingResultEvent = SparkListenerTaskGettingResult(task) - assert(filter.filterTaskGettingResult(taskGettingResultEvent) === Some(false)) - - val taskEndEvent = SparkListenerTaskEnd(stage1.stageId, 0, "taskType", - Success, task, new ExecutorMetrics, null) - assert(filter.filterTaskEnd(taskEndEvent) === Some(false)) - } + assertFilterTaskEvents(filter, tasksForStage1, stage1, Some(false)) // Verifying with live job 2 val rddsForStage2 = createRddsWithId(3 to 4) @@ -92,28 +79,16 @@ class BasicEventFilterSuite extends SparkFunSuite { assert(filter.filterJobStart(jobStartEventForJob2) === Some(true)) // stage events for live job should be filtered in - assert(filter.filterStageSubmitted(stageSubmittedEventsForJob2) === Some(true)) - assert(filter.filterStageCompleted(stageCompletedEventsForJob2) === Some(true)) - unpersistRDDEventsForJob2.foreach { event => - assert(filter.filterUnpersistRDD(event) === Some(true)) - } - - val taskSpeculativeTaskSubmittedEvent2 = SparkListenerSpeculativeTaskSubmitted(stage2.stageId, - stageAttemptId = 1) - assert(filter.filterSpeculativeTaskSubmitted(taskSpeculativeTaskSubmittedEvent2) === Some(true)) + assertFilterStageEvents( + filter, + stageSubmittedEventsForJob2, + stageCompletedEventsForJob2, + unpersistRDDEventsForJob2, + SparkListenerSpeculativeTaskSubmitted(stage2.stageId, stageAttemptId = 1), + Some(true)) // task events for live job should be filtered in - tasksForStage2.foreach { task => - val taskStartEvent = SparkListenerTaskStart(stage2.stageId, 0, task) - assert(filter.filterTaskStart(taskStartEvent) === Some(true)) - - val taskGettingResultEvent = SparkListenerTaskGettingResult(task) - assert(filter.filterTaskGettingResult(taskGettingResultEvent) === Some(true)) - - val taskEndEvent = SparkListenerTaskEnd(stage1.stageId, 0, "taskType", - Success, task, new ExecutorMetrics, null) - assert(filter.filterTaskEnd(taskEndEvent) === Some(true)) - } + assertFilterTaskEvents(filter, tasksForStage2, stage2, Some(true)) } test("filter out events for dead executors") { @@ -167,4 +142,46 @@ class BasicEventFilterSuite extends SparkFunSuite { assertNone(filter.filterNodeUnblacklisted(SparkListenerNodeUnblacklisted(0, "host1"))) assertNone(filter.filterOtherEvent(SparkListenerLogStart("testVersion"))) } + + private def assertFilterJobEvents( + filter: BasicEventFilter, + jobStart: SparkListenerJobStart, + jobEnd: SparkListenerJobEnd, + expectedVal: Option[Boolean]): Unit = { + assert(filter.filterJobStart(jobStart) === expectedVal) + assert(filter.filterJobEnd(jobEnd) === expectedVal) + } + + private def assertFilterStageEvents( + filter: BasicEventFilter, + stageSubmitted: SparkListenerStageSubmitted, + stageCompleted: SparkListenerStageCompleted, + unpersistRDDs: Seq[SparkListenerUnpersistRDD], + taskSpeculativeSubmitted: SparkListenerSpeculativeTaskSubmitted, + expectedVal: Option[Boolean]): Unit = { + assert(filter.filterStageSubmitted(stageSubmitted) === expectedVal) + assert(filter.filterStageCompleted(stageCompleted) === expectedVal) + unpersistRDDs.foreach { event => + assert(filter.filterUnpersistRDD(event) === expectedVal) + } + assert(filter.filterSpeculativeTaskSubmitted(taskSpeculativeSubmitted) === expectedVal) + } + + private def assertFilterTaskEvents( + filter: BasicEventFilter, + taskInfos: Seq[TaskInfo], + stageInfo: StageInfo, + expectedVal: Option[Boolean]): Unit = { + taskInfos.foreach { task => + val taskStartEvent = SparkListenerTaskStart(stageInfo.stageId, 0, task) + assert(filter.filterTaskStart(taskStartEvent) === expectedVal) + + val taskGettingResultEvent = SparkListenerTaskGettingResult(task) + assert(filter.filterTaskGettingResult(taskGettingResultEvent) === expectedVal) + + val taskEndEvent = SparkListenerTaskEnd(stageInfo.stageId, 0, "taskType", + Success, task, new ExecutorMetrics, null) + assert(filter.filterTaskEnd(taskEndEvent) === expectedVal) + } + } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala index 9a3f3784b774b..248eb7bcfa970 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala @@ -21,7 +21,7 @@ import java.io.File import scala.io.{Codec, Source} -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.json4s.jackson.JsonMethods.parse import org.apache.spark.{SparkConf, SparkFunSuite} @@ -66,20 +66,10 @@ class EventLogFileCompactorSuite extends SparkFunSuite { val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) val fileStatuses = logPaths.map { p => fs.getFileStatus(new Path(p)) } - val filesToRead = compactor.compact(fileStatuses) - - // 3 (max file to retain) + 1 (compacted file) - assert(filesToRead.length === 4) - val originalFilesToRead = filesToRead.takeRight(3) - val originFileToCompact = fileStatuses.takeRight(4).head.getPath - val compactFilePath = filesToRead.head.getPath - - assert(compactFilePath.getName === originFileToCompact.getName + EventLogFileWriter.COMPACTED) - assert(originalFilesToRead.map(_.getPath) === fileStatuses.takeRight(3).map(_.getPath)) + assertCompaction(fs, fileStatuses, compactor.compact(fileStatuses)) // compacted files will be removed fileStatuses.take(2).foreach { status => assert(!fs.exists(status.getPath)) } - filesToRead.foreach { status => assert(fs.exists(status.getPath)) } } } @@ -120,20 +110,10 @@ class EventLogFileCompactorSuite extends SparkFunSuite { val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) val fileStatuses = newLogPaths.map { p => fs.getFileStatus(new Path(p)) } - val filesToRead = compactor.compact(fileStatuses) - - // 3 (max file to retain) + 1 (compacted file) - assert(filesToRead.length === 4) - val originalFilesToRead = filesToRead.takeRight(3) - val originFileToCompact = fileStatuses.takeRight(4).head.getPath - val compactFilePath = filesToRead.head.getPath - - assert(compactFilePath.getName === originFileToCompact.getName + EventLogFileWriter.COMPACTED) - assert(originalFilesToRead.map(_.getPath) === fileStatuses.takeRight(3).map(_.getPath)) + assertCompaction(fs, fileStatuses, compactor.compact(fileStatuses)) // compacted files will be removed - we don't check files "before" the rightmost compact file fileStatuses.drop(2).dropRight(3).foreach { status => assert(!fs.exists(status.getPath)) } - filesToRead.foreach { status => assert(fs.exists(status.getPath)) } } } @@ -161,19 +141,33 @@ class EventLogFileCompactorSuite extends SparkFunSuite { val compactFilePath = filesToRead.head.getPath Utils.tryWithResource(EventLogFileReader.openEventLog(compactFilePath, fs)) { is => - val lines = Source.fromInputStream(is)(Codec.UTF8).getLines() - var linesLength = 0 + val lines = Source.fromInputStream(is)(Codec.UTF8).getLines().toList + assert(lines.length === 2, "Compacted file should have only two events being filtered in") lines.foreach { line => - linesLength += 1 val event = JsonProtocol.sparkEventFromJson(parse(line)) assert(!event.isInstanceOf[SparkListenerJobStart] && !event.isInstanceOf[SparkListenerJobEnd]) } - assert(linesLength === 2, "Compacted file should have only two events being filtered in") } } } + private def assertCompaction( + fs: FileSystem, + originalFiles: Seq[FileStatus], + retValue: Seq[FileStatus]): Unit = { + // 3 (max file to retain) + 1 (compacted file) + assert(retValue.length === 4) + val originalFilesToRead = retValue.takeRight(3) + val originFileToCompact = originalFiles.takeRight(4).head.getPath + val compactFilePath = retValue.head.getPath + + assert(compactFilePath.getName === originFileToCompact.getName + EventLogFileWriter.COMPACTED) + assert(originalFilesToRead.map(_.getPath) === originalFiles.takeRight(3).map(_.getPath)) + + retValue.foreach { status => assert(fs.exists(status.getPath)) } + } + private def writeDummyEventLogFile(dir: File, idx: Int): String = { EventLogTestHelper.writeEventLogFile(sparkConf, hadoopConf, dir, idx, Seq(SparkListenerApplicationStart("app", Some("app"), 0, "user", None))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala index 268402177413a..713e86361d531 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.history import scala.collection.mutable -import org.apache.spark.deploy.history.{EventFilter, EventFilterBuilder} +import org.apache.spark.deploy.history.{EventFilter, EventFilterBuilder, JobEventFilter} import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ import org.apache.spark.sql.execution.SQLExecution @@ -118,72 +118,56 @@ private[spark] class SQLEventFilterBuilder extends SparkListener with EventFilte */ private[spark] class SQLLiveEntitiesEventFilter( liveExecutionToJobs: Map[Long, Set[Int]], - jobToStages: Map[Int, Seq[Int]], - stageToTasks: Map[Int, Set[Long]], - stageToRDDs: Map[Int, Seq[Int]]) extends EventFilter with Logging { - - private val liveTasks: Set[Long] = stageToTasks.values match { - case xs if xs.isEmpty => Set.empty[Long] - case xs => xs.reduce(_ ++ _).toSet - } - - private val liveRDDs: Set[Int] = stageToRDDs.values match { - case xs if xs.isEmpty => Set.empty[Int] - case xs => xs.reduce(_ ++ _).toSet - } + _jobToStages: Map[Int, Seq[Int]], + _stageToTasks: Map[Int, Set[Long]], + _stageToRDDs: Map[Int, Seq[Int]]) + extends JobEventFilter(_jobToStages, _stageToTasks, _stageToRDDs) with Logging { if (log.isDebugEnabled) { logDebug(s"live executions : ${liveExecutionToJobs.keySet}") logDebug(s"jobs in live executions : ${liveExecutionToJobs.values.flatten}") - logDebug(s"jobs : ${jobToStages.keySet}") - logDebug(s"stages in jobs : ${jobToStages.values.flatten}") - logDebug(s"stages : ${stageToTasks.keySet}") - logDebug(s"tasks in stages : ${stageToTasks.values.flatten}") - logDebug(s"RDDs in stages : ${stageToRDDs.values.flatten}") } override def filterStageCompleted(event: SparkListenerStageCompleted): Option[Boolean] = { - trueOrNone(stageToTasks.contains(event.stageInfo.stageId)) + trueOrNone(super.filterStageCompleted(event)) } override def filterStageSubmitted(event: SparkListenerStageSubmitted): Option[Boolean] = { - trueOrNone(stageToTasks.contains(event.stageInfo.stageId)) + trueOrNone(super.filterStageSubmitted(event)) } override def filterTaskStart(event: SparkListenerTaskStart): Option[Boolean] = { - trueOrNone(liveTasks.contains(event.taskInfo.taskId)) + trueOrNone(super.filterTaskStart(event)) } override def filterTaskGettingResult(event: SparkListenerTaskGettingResult): Option[Boolean] = { - trueOrNone(liveTasks.contains(event.taskInfo.taskId)) + trueOrNone(super.filterTaskGettingResult(event)) } override def filterTaskEnd(event: SparkListenerTaskEnd): Option[Boolean] = { - trueOrNone(liveTasks.contains(event.taskInfo.taskId)) + trueOrNone(super.filterTaskEnd(event)) } override def filterJobStart(event: SparkListenerJobStart): Option[Boolean] = { - trueOrNone(jobToStages.contains(event.jobId)) + trueOrNone(super.filterJobStart(event)) } override def filterJobEnd(event: SparkListenerJobEnd): Option[Boolean] = { - trueOrNone(jobToStages.contains(event.jobId)) + trueOrNone(super.filterJobEnd(event)) } override def filterUnpersistRDD(event: SparkListenerUnpersistRDD): Option[Boolean] = { - trueOrNone(liveRDDs.contains(event.rddId)) + trueOrNone(super.filterUnpersistRDD(event)) } override def filterExecutorMetricsUpdate( event: SparkListenerExecutorMetricsUpdate): Option[Boolean] = { - trueOrNone(event.accumUpdates.exists { case (_, stageId, _, _) => - stageToTasks.contains(stageId) - }) + trueOrNone(super.filterExecutorMetricsUpdate(event)) } override def filterSpeculativeTaskSubmitted( event: SparkListenerSpeculativeTaskSubmitted): Option[Boolean] = { - trueOrNone(stageToTasks.contains(event.stageId)) + trueOrNone(super.filterSpeculativeTaskSubmitted(event)) } override def filterOtherEvent(event: SparkListenerEvent): Option[Boolean] = event match { @@ -214,11 +198,11 @@ private[spark] class SQLLiveEntitiesEventFilter( Some(liveExecutionToJobs.contains(event.executionId)) } - private def trueOrNone(booleanValue: Boolean): Option[Boolean] = { - if (booleanValue) { - Some(booleanValue) - } else { - None + private def trueOrNone(booleanValue: Option[Boolean]): Option[Boolean] = { + booleanValue match { + case Some(true) => Some(true) + case Some(false) => None + case None => None } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala index 7fca1a805f8f9..0c913e3a87476 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala @@ -32,56 +32,6 @@ class SQLEventFilterBuilderSuite extends SparkFunSuite { } test("track live SQL executions") { - case class JobInfo( - stageIds: Seq[Int], - stageToTaskIds: Map[Int, Seq[Long]], - stageToRddIds: Map[Int, Seq[Int]]) - - def pushJobEventsWithoutJobEnd( - listener: SQLEventFilterBuilder, - jobId: Int, - execIds: Array[String], - sqlExecId: Option[String], - time: Long): JobInfo = { - // Start a job with 1 stages / 4 tasks each - val rddsForStage = createRdds(2) - val stage = createStage(rddsForStage, Nil) - - val jobProps = createJobProps() - sqlExecId.foreach { id => jobProps.setProperty(SQLExecution.EXECUTION_ID_KEY, id) } - - listener.onJobStart(SparkListenerJobStart(jobId, time, Seq(stage), jobProps)) - - // Submit stage - stage.submissionTime = Some(time) - listener.onStageSubmitted(SparkListenerStageSubmitted(stage, jobProps)) - - // Start tasks from stage - val s1Tasks = ListenerEventsTestHelper.createTasks(4, execIds, time) - s1Tasks.foreach { task => - listener.onTaskStart(SparkListenerTaskStart(stage.stageId, - stage.attemptNumber(), task)) - } - - // Succeed all tasks in stage. - val s1Metrics = TaskMetrics.empty - s1Metrics.setExecutorCpuTime(2L) - s1Metrics.setExecutorRunTime(4L) - - s1Tasks.foreach { task => - task.markFinished(TaskState.FINISHED, time) - listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, - "taskType", Success, task, new ExecutorMetrics, s1Metrics)) - } - - // End stage. - stage.completionTime = Some(time) - listener.onStageCompleted(SparkListenerStageCompleted(stage)) - - JobInfo(Seq(stage.stageId), Map(stage.stageId -> s1Tasks.map(_.taskId)), - Map(stage.stageId -> rddsForStage.map(_.id))) - } - var time = 0L val listener = new SQLEventFilterBuilder @@ -158,4 +108,54 @@ class SQLEventFilterBuilderSuite extends SparkFunSuite { assert(listener.stageToTasks.isEmpty) assert(listener.stageToRDDs.isEmpty) } + + case class JobInfo( + stageIds: Seq[Int], + stageToTaskIds: Map[Int, Seq[Long]], + stageToRddIds: Map[Int, Seq[Int]]) + + private def pushJobEventsWithoutJobEnd( + listener: SQLEventFilterBuilder, + jobId: Int, + execIds: Array[String], + sqlExecId: Option[String], + time: Long): JobInfo = { + // Start a job with 1 stages / 4 tasks each + val rddsForStage = createRdds(2) + val stage = createStage(rddsForStage, Nil) + + val jobProps = createJobProps() + sqlExecId.foreach { id => jobProps.setProperty(SQLExecution.EXECUTION_ID_KEY, id) } + + listener.onJobStart(SparkListenerJobStart(jobId, time, Seq(stage), jobProps)) + + // Submit stage + stage.submissionTime = Some(time) + listener.onStageSubmitted(SparkListenerStageSubmitted(stage, jobProps)) + + // Start tasks from stage + val s1Tasks = ListenerEventsTestHelper.createTasks(4, execIds, time) + s1Tasks.foreach { task => + listener.onTaskStart(SparkListenerTaskStart(stage.stageId, + stage.attemptNumber(), task)) + } + + // Succeed all tasks in stage. + val s1Metrics = TaskMetrics.empty + s1Metrics.setExecutorCpuTime(2L) + s1Metrics.setExecutorRunTime(4L) + + s1Tasks.foreach { task => + task.markFinished(TaskState.FINISHED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, + "taskType", Success, task, new ExecutorMetrics, s1Metrics)) + } + + // End stage. + stage.completionTime = Some(time) + listener.onStageCompleted(SparkListenerStageCompleted(stage)) + + JobInfo(Seq(stage.stageId), Map(stage.stageId -> s1Tasks.map(_.taskId)), + Map(stage.stageId -> rddsForStage.map(_.id))) + } } From a3b7b1fb3864dc0946e469c970a8f418cbd12d32 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 15 Nov 2019 13:22:42 +0900 Subject: [PATCH 14/28] Reflect review comments --- .../history/BasicEventFilterBuilder.scala | 16 +++++------ .../history/EventLogFileCompactor.scala | 27 ++++++++++--------- .../deploy/history/EventLogFileReaders.scala | 9 ++++--- .../deploy/history/EventLogFileWriters.scala | 2 +- .../history/EventLogFileReadersSuite.scala | 6 +++-- .../history/EventLogFileWritersSuite.scala | 4 +-- .../history/SQLEventFilterBuilder.scala | 8 +++--- 7 files changed, 36 insertions(+), 36 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala index f8de6663b2bb4..90dbcab6492fb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala @@ -92,13 +92,11 @@ private[spark] abstract class JobEventFilter( case xs => xs.reduce(_ ++ _).toSet } - if (log.isDebugEnabled) { - logDebug(s"jobs : ${jobToStages.keySet}") - logDebug(s"stages in jobs : ${jobToStages.values.flatten}") - logDebug(s"stages : ${stageToTasks.keySet}") - logDebug(s"tasks in stages : ${stageToTasks.values.flatten}") - logDebug(s"RDDs in stages : ${stageToRDDs.values.flatten}") - } + logDebug(s"jobs : ${jobToStages.keySet}") + logDebug(s"stages in jobs : ${jobToStages.values.flatten}") + logDebug(s"stages : ${stageToTasks.keySet}") + logDebug(s"tasks in stages : ${stageToTasks.values.flatten}") + logDebug(s"RDDs in stages : ${stageToRDDs.values.flatten}") override def filterStageCompleted(event: SparkListenerStageCompleted): Option[Boolean] = { Some(stageToTasks.contains(event.stageInfo.stageId)) @@ -157,9 +155,7 @@ private[spark] class BasicEventFilter( liveExecutors: Set[String]) extends JobEventFilter(_liveJobToStages, _stageToTasks, _stageToRDDs) with Logging { - if (log.isDebugEnabled) { - logDebug(s"live executors : $liveExecutors") - } + logDebug(s"live executors : $liveExecutors") override def filterExecutorAdded(event: SparkListenerExecutorAdded): Option[Boolean] = { Some(liveExecutors.contains(event.executorId)) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index 08139e314ab35..4cb13b269083a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -23,8 +23,8 @@ import java.util.ServiceLoader import scala.collection.JavaConverters._ import scala.io.{Codec, Source} +import scala.util.control.NonFatal -import com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.json4s.jackson.JsonMethods.parse @@ -158,24 +158,27 @@ class FilteredEventLogFileRewriter( try { val lineEntries = lines.zipWithIndex while (lineEntries.hasNext) { - try { - val entry = lineEntries.next() + val entry = lineEntries.next() - currentLine = entry._1 - lineNumber = entry._2 + 1 + currentLine = entry._1 + lineNumber = entry._2 + 1 - val event = JsonProtocol.sparkEventFromJson(parse(currentLine)) - if (checkFilters(event)) { - logWriter.writeLine(currentLine) - } + val event = try { + Some(JsonProtocol.sparkEventFromJson(parse(currentLine))) } catch { // ignore any exception occurred from unidentified json // just skip handling and write the line - case _: ClassNotFoundException => logWriter.writeLine(currentLine) - case _: UnrecognizedPropertyException => logWriter.writeLine(currentLine) + case NonFatal(_) => + logWriter.writeLine(currentLine) + None + } + + event.foreach { e => + if (checkFilters(e)) { + logWriter.writeLine(currentLine) + } } } - true } catch { case e: Exception => logError(s"Exception parsing Spark event log: ${fileStatus.getPath.getName}", e) 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 bcaa8d0901f8c..301d44bfe042f 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 @@ -220,7 +220,7 @@ class RollingEventLogFilesFileReader( private lazy val eventLogFiles: Seq[FileStatus] = { val eventLogFiles = files.filter(isEventLogFile).sortBy { status => val filePath = status.getPath - var idx = getIndex(filePath.getName).toDouble + var idx = getEventLogFileIndex(filePath.getName).toDouble // trick to place compacted file later than normal file if index is same. if (EventLogFileWriter.isCompacted(filePath)) { idx += 0.1 @@ -229,13 +229,14 @@ class RollingEventLogFilesFileReader( } val filesToRead = RollingEventLogFilesFileReader.dropBeforeLastCompactFile(eventLogFiles) - val indices = filesToRead.map { file => getIndex(file.getPath.getName) } + val indices = filesToRead.map { file => getEventLogFileIndex(file.getPath.getName) } require((indices.head to indices.last) == indices, "Found missing event log file, expected" + - s" indices: ${(indices.head to indices.last)}, actual: ${indices}") + s" indices: ${indices.head to indices.last}, actual: ${indices}") filesToRead } - override def lastIndex: Option[Long] = Some(getIndex(lastEventLogFile.getPath.getName)) + override def lastIndex: Option[Long] = Some( + getEventLogFileIndex(lastEventLogFile.getPath.getName)) override def fileSizeForLastIndex: Long = lastEventLogFile.getLen 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 8b497f4cbea93..3111d303653b1 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 @@ -414,7 +414,7 @@ object RollingEventLogFilesWriter { status.isFile && status.getPath.getName.startsWith(APPSTATUS_FILE_NAME_PREFIX) } - def getIndex(eventLogFileName: String): Long = { + def getEventLogFileIndex(eventLogFileName: String): Long = { require(isEventLogFile(eventLogFileName), "Not an event log file!") val index = eventLogFileName.stripPrefix(EVENT_LOG_FILE_NAME_PREFIX).split("_")(0) index.toLong 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 a2ce4acdaaf37..8eab2da1a37b7 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 @@ -288,13 +288,15 @@ class RollingEventLogFilesReaderSuite extends EventLogFileReadersSuite { assert(status.isDirectory) val statusInDir = fileSystem.listStatus(logPath) - val eventFiles = statusInDir.filter(isEventLogFile).sortBy { s => getIndex(s.getPath.getName) } + val eventFiles = statusInDir.filter(isEventLogFile).sortBy { s => + getEventLogFileIndex(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(getIndex(lastEventFile.getPath.getName))) + assert(reader.lastIndex === Some(getEventLogFileIndex(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 c4b40884eebf5..060b878fb8ef2 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 @@ -291,7 +291,7 @@ class RollingEventLogFilesWriterSuite extends EventLogFileWritersSuite { expectedMaxSizeBytes: Long): Unit = { assert(eventLogFiles.forall(f => f.getLen <= expectedMaxSizeBytes)) assert((1 to expectedLastIndex) === - eventLogFiles.map(f => getIndex(f.getPath.getName))) + eventLogFiles.map(f => getEventLogFileIndex(f.getPath.getName))) } val appId = getUniqueApplicationId @@ -373,6 +373,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 => getEventLogFileIndex(fs.getPath.getName) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala index 713e86361d531..82e15bd80d122 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala @@ -123,10 +123,8 @@ private[spark] class SQLLiveEntitiesEventFilter( _stageToRDDs: Map[Int, Seq[Int]]) extends JobEventFilter(_jobToStages, _stageToTasks, _stageToRDDs) with Logging { - if (log.isDebugEnabled) { - logDebug(s"live executions : ${liveExecutionToJobs.keySet}") - logDebug(s"jobs in live executions : ${liveExecutionToJobs.values.flatten}") - } + logDebug(s"live executions : ${liveExecutionToJobs.keySet}") + logDebug(s"jobs in live executions : ${liveExecutionToJobs.values.flatten}") override def filterStageCompleted(event: SparkListenerStageCompleted): Option[Boolean] = { trueOrNone(super.filterStageCompleted(event)) @@ -176,7 +174,7 @@ private[spark] class SQLLiveEntitiesEventFilter( case e: SparkListenerSQLExecutionEnd => filterExecutionEnd(e) case e: SparkListenerDriverAccumUpdates => filterDriverAccumUpdates(e) - // these events are for finished batches so safer to ignore + // these events are for finished batches so safer to ignore case _: StreamingQueryListener.QueryProgressEvent => Some(false) case _ => None } From a3a0c4acfdd602ad0a3566cc49ac38b707603fe7 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 15 Nov 2019 13:40:33 +0900 Subject: [PATCH 15/28] More comments reflection --- .../deploy/history/EventLogFileCompactor.scala | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index 4cb13b269083a..9e4ecf1e259ef 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -137,9 +137,8 @@ class FilteredEventLogFileRewriter( require(eventLogFiles.nonEmpty) val targetEventLogFilePath = eventLogFiles.last.getPath - val logWriter: CompactedEventLogFileWriter = new CompactedEventLogFileWriter( - targetEventLogFilePath, "dummy", None, targetEventLogFilePath.getParent.toUri, - sparkConf, hadoopConf) + val logWriter = new CompactedEventLogFileWriter(targetEventLogFilePath, "dummy", None, + targetEventLogFilePath.getParent.toUri, sparkConf, hadoopConf) logWriter.start() eventLogFiles.foreach { file => rewriteFile(logWriter, file) } @@ -169,13 +168,13 @@ class FilteredEventLogFileRewriter( // ignore any exception occurred from unidentified json // just skip handling and write the line case NonFatal(_) => - logWriter.writeLine(currentLine) + logWriter.writeEvent(currentLine, flushLogger = true) None } event.foreach { e => if (checkFilters(e)) { - logWriter.writeLine(currentLine) + logWriter.writeEvent(currentLine, flushLogger = true) } } } @@ -245,9 +244,4 @@ class CompactedEventLogFileWriter( extends SingleEventLogFileWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) { override val logPath: String = originalFilePath.toUri.toString + EventLogFileWriter.COMPACTED - - // override to make writeLine method be 'public' only for this class - override def writeLine(line: String, flushLogger: Boolean): Unit = { - super.writeLine(line, flushLogger) - } } From 7054b2c993e7c49cc7dc6009cc60c86fadd07815 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 18 Nov 2019 08:16:35 +0900 Subject: [PATCH 16/28] Fix indentation --- docs/configuration.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index e703cf2b6d180..9cc8696b312db 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1028,10 +1028,10 @@ Apart from these, the following properties are also available, and may be useful Int.MaxValue The maximum number of event log files which will be retained as non-compacted. - By default, all event log files will be retained. Please set the configuration and - spark.eventLog.rolling.maxFileSize accordingly if you want to control - the overall size of event log files. The event log files older than these retained - files will be compacted into single file and deleted afterwards. + By default, all event log files will be retained. Please set the configuration and + spark.eventLog.rolling.maxFileSize accordingly if you want to control + the overall size of event log files. The event log files older than these retained + files will be compacted into single file and deleted afterwards. From 5a54af967b32739c4cbfe9e1f1c4ba7b085437b3 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 25 Nov 2019 23:08:35 +0900 Subject: [PATCH 17/28] Introduce the rate of filter to determine whether the event log for app is eligible for compaction or not --- .../spark/deploy/history/EventFilter.scala | 128 +++++++++++++++ .../history/EventLogFileCompactor.scala | 119 ++------------ .../EventLogFilterRateCalculator.scala | 79 +++++++++ .../deploy/history/FsHistoryProvider.scala | 91 +++++++++-- .../EventLogFilterRateCalculatorSuite.scala | 68 ++++++++ .../deploy/history/EventLogTestHelper.scala | 42 ++++- .../FilteredEventLogFileRewriterSuite.scala | 41 +---- .../history/FsHistoryProviderSuite.scala | 154 ++++++++++++++++-- .../status/ListenerEventsTestHelper.scala | 51 +++++- .../history/SQLEventFilterBuilderSuite.scala | 70 ++------ 10 files changed, 614 insertions(+), 229 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculator.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculatorSuite.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala index 5fbd9951af559..72137a228bfd4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala @@ -17,7 +17,18 @@ package org.apache.spark.deploy.history +import java.util.ServiceLoader + +import scala.collection.JavaConverters._ +import scala.io.{Codec, Source} +import scala.util.control.NonFatal + +import org.apache.hadoop.fs.{FileSystem, Path} +import org.json4s.jackson.JsonMethods.parse + +import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ +import org.apache.spark.util.{JsonProtocol, Utils} /** * EventFilterBuilder provides the interface to gather the information from events being received @@ -28,6 +39,28 @@ private[spark] trait EventFilterBuilder extends SparkListenerInterface { def createFilter(): EventFilter } +object EventFilterBuilder { + /** + * Loads all available EventFilterBuilders in classloader via ServiceLoader, and initializes + * them via replaying events in given files. + */ + def initializeBuilders(fs: FileSystem, files: Seq[Path]): Seq[EventFilterBuilder] = { + val bus = new ReplayListenerBus() + + val builders = ServiceLoader.load(classOf[EventFilterBuilder], + Utils.getContextOrSparkClassLoader).asScala.toSeq + builders.foreach(bus.addListener) + + files.foreach { log => + Utils.tryWithResource(EventLogFileReader.openEventLog(log, fs)) { in => + bus.replay(in, log.getName) + } + } + + builders + } +} + /** * [[EventFilter]] decides whether the given event should be filtered in, or filtered out when * compacting event log files. @@ -96,3 +129,98 @@ private[spark] trait EventFilter { def filterOtherEvent(event: SparkListenerEvent): Option[Boolean] = None } +object EventFilter { + def checkFilters(filters: Seq[EventFilter], event: SparkListenerEvent): Boolean = { + val results = filters.flatMap(filter => applyFilter(filter, event)) + results.isEmpty || results.forall(_ == true) + } + + private def applyFilter(filter: EventFilter, event: SparkListenerEvent): Option[Boolean] = { + // This pattern match should have same list of event types, but it would be safe even if + // it's out of sync, once filter doesn't mark events to filter out for unknown event types. + event match { + case event: SparkListenerStageSubmitted => filter.filterStageSubmitted(event) + case event: SparkListenerStageCompleted => filter.filterStageCompleted(event) + case event: SparkListenerJobStart => filter.filterJobStart(event) + case event: SparkListenerJobEnd => filter.filterJobEnd(event) + case event: SparkListenerTaskStart => filter.filterTaskStart(event) + case event: SparkListenerTaskGettingResult => filter.filterTaskGettingResult(event) + case event: SparkListenerTaskEnd => filter.filterTaskEnd(event) + case event: SparkListenerEnvironmentUpdate => filter.filterEnvironmentUpdate(event) + case event: SparkListenerBlockManagerAdded => filter.filterBlockManagerAdded(event) + case event: SparkListenerBlockManagerRemoved => filter.filterBlockManagerRemoved(event) + case event: SparkListenerUnpersistRDD => filter.filterUnpersistRDD(event) + case event: SparkListenerApplicationStart => filter.filterApplicationStart(event) + case event: SparkListenerApplicationEnd => filter.filterApplicationEnd(event) + case event: SparkListenerExecutorMetricsUpdate => filter.filterExecutorMetricsUpdate(event) + case event: SparkListenerStageExecutorMetrics => filter.filterStageExecutorMetrics(event) + case event: SparkListenerExecutorAdded => filter.filterExecutorAdded(event) + case event: SparkListenerExecutorRemoved => filter.filterExecutorRemoved(event) + case event: SparkListenerExecutorBlacklistedForStage => + filter.filterExecutorBlacklistedForStage(event) + case event: SparkListenerNodeBlacklistedForStage => + filter.filterNodeBlacklistedForStage(event) + case event: SparkListenerExecutorBlacklisted => filter.filterExecutorBlacklisted(event) + case event: SparkListenerExecutorUnblacklisted => filter.filterExecutorUnblacklisted(event) + case event: SparkListenerNodeBlacklisted => filter.filterNodeBlacklisted(event) + case event: SparkListenerNodeUnblacklisted => filter.filterNodeUnblacklisted(event) + case event: SparkListenerBlockUpdated => filter.filterBlockUpdated(event) + case event: SparkListenerSpeculativeTaskSubmitted => + filter.filterSpeculativeTaskSubmitted(event) + case _ => filter.filterOtherEvent(event) + } + } +} + +trait EventFilterApplier extends Logging { + val fs: FileSystem + val filters: Seq[EventFilter] + + def applyFilter(path: Path): Unit = { + Utils.tryWithResource(EventLogFileReader.openEventLog(path, fs)) { in => + val lines = Source.fromInputStream(in)(Codec.UTF8).getLines() + + var currentLine: String = null + var lineNumber: Int = 0 + + try { + val lineEntries = lines.zipWithIndex + while (lineEntries.hasNext) { + val entry = lineEntries.next() + + currentLine = entry._1 + lineNumber = entry._2 + 1 + + val event = try { + Some(JsonProtocol.sparkEventFromJson(parse(currentLine))) + } catch { + // ignore any exception occurred from unidentified json + // just skip handling and write the line + case NonFatal(_) => + handleUnidentifiedLine(currentLine) + None + } + + event.foreach { e => + if (EventFilter.checkFilters(filters, e)) { + handleFilteredInEvent(currentLine, e) + } else { + handleFilteredOutEvent(currentLine, e) + } + } + } + } catch { + case e: Exception => + logError(s"Exception parsing Spark event log: ${path.getName}", e) + logError(s"Malformed line #$lineNumber: $currentLine\n") + throw e + } + } + } + + protected def handleFilteredInEvent(line: String, event: SparkListenerEvent): Unit + + protected def handleFilteredOutEvent(line: String, event: SparkListenerEvent): Unit + + protected def handleUnidentifiedLine(line: String): Unit +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index 9e4ecf1e259ef..8a190bc4c3fb0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -19,21 +19,14 @@ package org.apache.spark.deploy.history import java.io.IOException import java.net.URI -import java.util.ServiceLoader - -import scala.collection.JavaConverters._ -import scala.io.{Codec, Source} -import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} -import org.json4s.jackson.JsonMethods.parse import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.internal.config.EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN import org.apache.spark.scheduler._ -import org.apache.spark.util.{JsonProtocol, Utils} /** * This class compacts the old event log files into one compact file, via two phases reading: @@ -74,17 +67,7 @@ class EventLogFileCompactor( if (filesToCompact.isEmpty) { filesToRetain } else { - val bus = new ReplayListenerBus() - - val builders = ServiceLoader.load(classOf[EventFilterBuilder], - Utils.getContextOrSparkClassLoader).asScala.toSeq - builders.foreach(bus.addListener) - - filesToCompact.foreach { log => - Utils.tryWithResource(EventLogFileReader.openEventLog(log.getPath, fs)) { in => - bus.replay(in, log.getPath.getName) - } - } + val builders = EventFilterBuilder.initializeBuilders(fs, filesToCompact.map(_.getPath)) val rewriter = new FilteredEventLogFileRewriter(sparkConf, hadoopConf, fs, builders.map(_.createFilter())) @@ -130,102 +113,34 @@ class EventLogFileCompactor( class FilteredEventLogFileRewriter( sparkConf: SparkConf, hadoopConf: Configuration, - fs: FileSystem, - filters: Seq[EventFilter]) extends Logging { + override val fs: FileSystem, + override val filters: Seq[EventFilter]) extends EventFilterApplier { + + private var logWriter: Option[CompactedEventLogFileWriter] = None def rewrite(eventLogFiles: Seq[FileStatus]): String = { require(eventLogFiles.nonEmpty) val targetEventLogFilePath = eventLogFiles.last.getPath - val logWriter = new CompactedEventLogFileWriter(targetEventLogFilePath, "dummy", None, - targetEventLogFilePath.getParent.toUri, sparkConf, hadoopConf) + logWriter = Some(new CompactedEventLogFileWriter(targetEventLogFilePath, "dummy", None, + targetEventLogFilePath.getParent.toUri, sparkConf, hadoopConf)) - logWriter.start() - eventLogFiles.foreach { file => rewriteFile(logWriter, file) } - logWriter.stop() + val writer = logWriter.get + writer.start() + eventLogFiles.foreach { file => applyFilter(file.getPath) } + writer.stop() - logWriter.logPath + writer.logPath } - private def rewriteFile(logWriter: CompactedEventLogFileWriter, fileStatus: FileStatus): Unit = { - Utils.tryWithResource(EventLogFileReader.openEventLog(fileStatus.getPath, fs)) { in => - val lines = Source.fromInputStream(in)(Codec.UTF8).getLines() - - var currentLine: String = null - var lineNumber: Int = 0 - - try { - val lineEntries = lines.zipWithIndex - while (lineEntries.hasNext) { - val entry = lineEntries.next() - - currentLine = entry._1 - lineNumber = entry._2 + 1 - - val event = try { - Some(JsonProtocol.sparkEventFromJson(parse(currentLine))) - } catch { - // ignore any exception occurred from unidentified json - // just skip handling and write the line - case NonFatal(_) => - logWriter.writeEvent(currentLine, flushLogger = true) - None - } - - event.foreach { e => - if (checkFilters(e)) { - logWriter.writeEvent(currentLine, flushLogger = true) - } - } - } - } catch { - case e: Exception => - logError(s"Exception parsing Spark event log: ${fileStatus.getPath.getName}", e) - logError(s"Malformed line #$lineNumber: $currentLine\n") - throw e - } - } + override protected def handleFilteredInEvent(line: String, event: SparkListenerEvent): Unit = { + logWriter.foreach { writer => writer.writeEvent(line, flushLogger = true) } } - private def checkFilters(event: SparkListenerEvent): Boolean = { - val results = filters.flatMap(filter => applyFilter(filter, event)) - results.isEmpty || results.forall(_ == true) - } + override protected def handleFilteredOutEvent(line: String, event: SparkListenerEvent): Unit = {} - private def applyFilter(filter: EventFilter, event: SparkListenerEvent): Option[Boolean] = { - // This pattern match should have same list of event types, but it would be safe even if - // it's out of sync, once filter doesn't mark events to filter out for unknown event types. - event match { - case event: SparkListenerStageSubmitted => filter.filterStageSubmitted(event) - case event: SparkListenerStageCompleted => filter.filterStageCompleted(event) - case event: SparkListenerJobStart => filter.filterJobStart(event) - case event: SparkListenerJobEnd => filter.filterJobEnd(event) - case event: SparkListenerTaskStart => filter.filterTaskStart(event) - case event: SparkListenerTaskGettingResult => filter.filterTaskGettingResult(event) - case event: SparkListenerTaskEnd => filter.filterTaskEnd(event) - case event: SparkListenerEnvironmentUpdate => filter.filterEnvironmentUpdate(event) - case event: SparkListenerBlockManagerAdded => filter.filterBlockManagerAdded(event) - case event: SparkListenerBlockManagerRemoved => filter.filterBlockManagerRemoved(event) - case event: SparkListenerUnpersistRDD => filter.filterUnpersistRDD(event) - case event: SparkListenerApplicationStart => filter.filterApplicationStart(event) - case event: SparkListenerApplicationEnd => filter.filterApplicationEnd(event) - case event: SparkListenerExecutorMetricsUpdate => filter.filterExecutorMetricsUpdate(event) - case event: SparkListenerStageExecutorMetrics => filter.filterStageExecutorMetrics(event) - case event: SparkListenerExecutorAdded => filter.filterExecutorAdded(event) - case event: SparkListenerExecutorRemoved => filter.filterExecutorRemoved(event) - case event: SparkListenerExecutorBlacklistedForStage => - filter.filterExecutorBlacklistedForStage(event) - case event: SparkListenerNodeBlacklistedForStage => - filter.filterNodeBlacklistedForStage(event) - case event: SparkListenerExecutorBlacklisted => filter.filterExecutorBlacklisted(event) - case event: SparkListenerExecutorUnblacklisted => filter.filterExecutorUnblacklisted(event) - case event: SparkListenerNodeBlacklisted => filter.filterNodeBlacklisted(event) - case event: SparkListenerNodeUnblacklisted => filter.filterNodeUnblacklisted(event) - case event: SparkListenerBlockUpdated => filter.filterBlockUpdated(event) - case event: SparkListenerSpeculativeTaskSubmitted => - filter.filterSpeculativeTaskSubmitted(event) - case _ => filter.filterOtherEvent(event) - } + override protected def handleUnidentifiedLine(line: String): Unit = { + logWriter.foreach { writer => writer.writeEvent(line, flushLogger = true) } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculator.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculator.scala new file mode 100644 index 0000000000000..9d1396641525f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculator.scala @@ -0,0 +1,79 @@ +/* + * 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.{FileSystem, Path} + +import org.apache.spark.scheduler.SparkListenerEvent + +/** + * This class calculates the rate of events being filtered-in, via two phases reading: + * + * 1) Initialize available [[EventFilterBuilder]] instances, and replay the event log files with + * builders, so that these builders can gather the information to create [[EventFilter]] instances. + * 2) Initialize [[EventFilter]] instances from [[EventFilterBuilder]] instances, and replay the + * event log files with filters. Counts the number of events vs filtered-in events and calculate + * the rate. + */ +class EventLogFilterRateCalculator(fs: FileSystem) { + def calculate(eventLogPaths: Seq[Path]): Double = { + val builders = EventFilterBuilder.initializeBuilders(fs, eventLogPaths) + doCalculate(eventLogPaths, builders.map(_.createFilter())) + } + + /** + * Exposed for tests - enable UTs to simply inject EventFilters instead of loading from + * ServiceLoader which is subject to change on which modules are available on classloader. + */ + private[spark] def doCalculate(eventLogPaths: Seq[Path], filters: Seq[EventFilter]): Double = { + val calc = new Calculator(fs, filters) + calc.calculate(eventLogPaths) + } + + private class Calculator( + override val fs: FileSystem, + override val filters: Seq[EventFilter]) extends EventFilterApplier { + private var allEvents = 0L + private var filteredInEvents = 0L + + def calculate(eventLogPaths: Seq[Path]): Double = { + clearValues() + eventLogPaths.foreach { path => applyFilter(path) } + filteredInEvents.toDouble / allEvents + } + + protected def handleFilteredInEvent(line: String, event: SparkListenerEvent): Unit = { + allEvents += 1 + filteredInEvents += 1 + } + + protected def handleFilteredOutEvent(line: String, event: SparkListenerEvent): Unit = { + allEvents += 1 + } + + protected def handleUnidentifiedLine(line: String): Unit = { + allEvents += 1 + filteredInEvents += 1 + } + + private def clearValues(): Unit = { + allEvents = 0L + filteredInEvents = 0L + } + } +} 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 b5b0816fc39d3..652b8bfc1d35a 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 @@ -162,6 +162,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private val fileCompactor = new EventLogFileCompactor(conf, hadoopConf, fs) + private val eventFilterRateCalculator = new EventLogFilterRateCalculator(fs) + private val blacklist = new ConcurrentHashMap[String, Long] // Visible for testing @@ -474,6 +476,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) attempt.logPath, attempt.fileSize, attempt.lastIndex, + attempt.compactible, attempt.adminAcls, attempt.viewAcls, attempt.adminAclsGroups, @@ -500,7 +503,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // 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(reader.rootPath.toString(), newLastScanTime, LogType.EventLogs, - None, None, reader.fileSizeForLastIndex, reader.lastIndex, + None, None, reader.fileSizeForLastIndex, reader.lastIndex, None, reader.completed)) reader.fileSizeForLastIndex > 0 } @@ -515,8 +518,10 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val tasks = updated.flatMap { entry => try { - val task: Future[Unit] = replayExecutor.submit( - () => mergeApplicationListing(entry, newLastScanTime, true)) + val task: Future[Unit] = replayExecutor.submit(() => { + val compactible = getOrUpdateCompactible(entry) + mergeApplicationListing(entry, newLastScanTime, true, compactible) + }) Some(task -> entry.rootPath) } catch { // let the iteration over the updated entries break, since an exception on @@ -665,13 +670,49 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } + private[spark] def getOrUpdateCompactible(reader: EventLogFileReader): Option[Boolean] = { + try { + val info = listing.read(classOf[LogInfo], reader.rootPath.toString) + val compactible = checkEligibilityForCompaction(info, reader) + if (info.compactible != compactible) { + listing.write(info.copy(compactible = compactible)) + } + compactible + } catch { + case _: NoSuchElementException => None + } + } + + protected def checkEligibilityForCompaction( + info: LogInfo, + reader: EventLogFileReader): Option[Boolean] = { + info.compactible.orElse { + // This is not applied to single event log file. + if (reader.lastIndex.isEmpty) { + Some(false) + } else { + if (reader.listEventLogFiles.length > 1) { + // We have at least one 'complete' file to check whether the event log is eligible to + // compact further. + val rate = eventFilterRateCalculator.calculate( + Seq(reader.listEventLogFiles.head.getPath)) + Some(rate < COMPACT_THRESHOLD_FILTER_IN_RATE) + } else { + // still cannot decide + None + } + } + } + } + /** * Replay the given log file, saving the application in the listing db. */ protected def mergeApplicationListing( reader: EventLogFileReader, scanTime: Long, - enableOptimizations: Boolean): Unit = { + enableOptimizations: Boolean, + compactible: Option[Boolean]): Unit = { val eventsFilter: ReplayEventsFilter = { eventString => eventString.startsWith(APPL_START_EVENT_PREFIX) || eventString.startsWith(APPL_END_EVENT_PREFIX) || @@ -690,7 +731,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) ((!appCompleted && fastInProgressParsing) || reparseChunkSize > 0) val bus = new ReplayListenerBus() - val listener = new AppListingListener(reader, clock, shouldHalt) + val listener = new AppListingListener(reader, compactible, clock, shouldHalt) bus.addListener(listener) logInfo(s"Parsing $logPath for listing data...") @@ -749,8 +790,8 @@ 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, reader.fileSizeForLastIndex, - reader.lastIndex, reader.completed)) + app.attempts.head.info.attemptId, reader.fileSizeForLastIndex, reader.lastIndex, + app.attempts.head.compactible, reader.completed)) // For a finished log, remove the corresponding "in progress" entry from the listing DB if // the file is really gone. @@ -774,7 +815,7 @@ 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(reader, scanTime, enableOptimizations = false) + mergeApplicationListing(reader, scanTime, enableOptimizations = false, compactible) case _ => // If the app hasn't written down its app ID to the logs, still record the entry in the @@ -782,7 +823,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // does not make progress after the configured max log age. listing.write( LogInfo(logPath.toString(), scanTime, LogType.EventLogs, None, None, - reader.fileSizeForLastIndex, reader.lastIndex, reader.completed)) + reader.fileSizeForLastIndex, reader.lastIndex, compactible, reader.completed)) } } @@ -917,7 +958,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, false)) + None, f.getLen(), None, None, false)) false } if (deleteFile) { @@ -949,7 +990,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private[spark] def rebuildAppStore( store: KVStore, reader: EventLogFileReader, - lastUpdated: Long): Unit = { + lastUpdated: Long, + compactible: Option[Boolean]): 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. val replayConf = conf.clone().set(ASYNC_TRACKING_ENABLED, false) @@ -965,8 +1007,13 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } replayBus.addListener(listener) try { - logInfo(s"Compacting ${reader.rootPath}...") - val newEventLogFiles = fileCompactor.compact(reader.listEventLogFiles) + val eventLogFiles = reader.listEventLogFiles + val newEventLogFiles = if (compactible.contains(true)) { + logInfo(s"Compacting ${reader.rootPath}...") + fileCompactor.compact(eventLogFiles) + } else { + eventLogFiles + } logInfo(s"Parsing ${reader.rootPath} to re-build UI...") parseAppEventLogs(newEventLogFiles, replayBus, !reader.completed) trackingStore.close(false) @@ -1086,7 +1133,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val lease = dm.lease(reader.totalSize, isCompressed) val newStorePath = try { Utils.tryWithResource(KVUtils.open(lease.tmpPath, metadata)) { store => - rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime()) + rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime(), attempt.compactible) } lease.commit(appId, attempt.info.attemptId) } catch { @@ -1102,7 +1149,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val store = new InMemoryStore() val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), attempt.lastIndex) - rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime()) + rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime(), attempt.compactible) store } @@ -1150,6 +1197,8 @@ private[history] object FsHistoryProvider { * all data and re-generate the listing data from the event logs. */ private[history] val CURRENT_LISTING_VERSION = 1L + + private val COMPACT_THRESHOLD_FILTER_IN_RATE = 0.3d } private[history] case class FsHistoryProviderMetadata( @@ -1175,6 +1224,7 @@ private[history] case class LogInfo( fileSize: Long, @JsonDeserialize(contentAs = classOf[JLong]) lastIndex: Option[Long], + compactible: Option[Boolean], isComplete: Boolean) private[history] class AttemptInfoWrapper( @@ -1183,6 +1233,7 @@ private[history] class AttemptInfoWrapper( val fileSize: Long, @JsonDeserialize(contentAs = classOf[JLong]) val lastIndex: Option[Long], + val compactible: Option[Boolean], val adminAcls: Option[String], val viewAcls: Option[String], val adminAclsGroups: Option[String], @@ -1207,12 +1258,13 @@ private[history] class ApplicationInfoWrapper( private[history] class AppListingListener( reader: EventLogFileReader, + compactible: Option[Boolean], clock: Clock, haltEnabled: Boolean) extends SparkListener { private val app = new MutableApplicationInfo() private val attempt = new MutableAttemptInfo(reader.rootPath.getName(), - reader.fileSizeForLastIndex, reader.lastIndex) + reader.fileSizeForLastIndex, reader.lastIndex, compactible) private var gotEnvUpdate = false private var halted = false @@ -1297,7 +1349,11 @@ private[history] class AppListingListener( } - private class MutableAttemptInfo(logPath: String, fileSize: Long, lastIndex: Option[Long]) { + private class MutableAttemptInfo( + logPath: String, + fileSize: Long, + lastIndex: Option[Long], + compactible: Option[Boolean]) { var attemptId: Option[String] = None var startTime = new Date(-1) var endTime = new Date(-1) @@ -1327,6 +1383,7 @@ private[history] class AppListingListener( logPath, fileSize, lastIndex, + compactible, adminAcls, viewAcls, adminAclsGroups, diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculatorSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculatorSuite.scala new file mode 100644 index 0000000000000..304783450bf1e --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculatorSuite.scala @@ -0,0 +1,68 @@ +/* + * 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 scala.collection.mutable + +import org.apache.hadoop.fs.Path + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.EventLogTestHelper.{TestEventFilter1, TestEventFilter2} +import org.apache.spark.scheduler.{SparkListenerApplicationEnd, SparkListenerApplicationStart, SparkListenerBlockManagerAdded, SparkListenerEvent, SparkListenerExecutorAdded, SparkListenerJobStart, SparkListenerNodeBlacklisted, SparkListenerNodeUnblacklisted, SparkListenerTaskStart, SparkListenerUnpersistRDD} +import org.apache.spark.storage.BlockManagerId + +class EventLogFilterRateCalculatorSuite extends SparkFunSuite { + private val sparkConf = new SparkConf() + private val hadoopConf = SparkHadoopUtil.newConfiguration(sparkConf) + + test("calculate filter-in rate") { + withTempDir { dir => + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + val events = new mutable.ArrayBuffer[SparkListenerEvent] + + // filterApplicationEnd: Some(true) & Some(true) => filter in + events += SparkListenerApplicationEnd(0) + + // filterBlockManagerAdded: Some(true) & Some(false) => filter out + events += SparkListenerBlockManagerAdded(0, BlockManagerId("1", "host1", 1), 10) + + // filterApplicationStart: Some(false) & Some(false) => filter out + events += SparkListenerApplicationStart("app", None, 0, "user", None) + + // filterNodeBlacklisted: None & Some(true) => filter in + events += SparkListenerNodeBlacklisted(0, "host1", 1) + + // filterNodeUnblacklisted: None & Some(false) => filter out + events += SparkListenerNodeUnblacklisted(0, "host1") + + // other events: None & None => filter in + events += SparkListenerUnpersistRDD(0) + + val logPath = EventLogTestHelper.writeEventLogFile(sparkConf, hadoopConf, dir, 1, events) + val logPath2 = EventLogTestHelper.writeEventLogFile(sparkConf, hadoopConf, dir, 2, events) + val logs = Seq(logPath, logPath2).map(new Path(_)) + + val filters = Seq(new TestEventFilter1, new TestEventFilter2) + val calculator = new EventLogFilterRateCalculator(fs) + // 6 filtered in, 6 filtered out + assert(0.5d === calculator.doCalculate(logs, filters)) + } + } +} 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 80cc1fc842ed1..0b16121f20249 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 @@ -26,7 +26,7 @@ import org.json4s.jackson.JsonMethods.{compact, render} import org.apache.spark.SparkConf import org.apache.spark.internal.config._ -import org.apache.spark.scheduler.SparkListenerEvent +import org.apache.spark.scheduler.{SparkListenerApplicationEnd, SparkListenerApplicationStart, SparkListenerBlockManagerAdded, SparkListenerEnvironmentUpdate, SparkListenerEvent, SparkListenerNodeBlacklisted, SparkListenerNodeUnblacklisted} import org.apache.spark.util.JsonProtocol object EventLogTestHelper { @@ -80,4 +80,44 @@ object EventLogTestHelper { def convertEvent(event: SparkListenerEvent): String = { compact(render(JsonProtocol.sparkEventToJson(event))) } + + class TestEventFilter1 extends EventFilter { + override def filterApplicationEnd(event: SparkListenerApplicationEnd): Option[Boolean] = { + Some(true) + } + + override def filterBlockManagerAdded(event: SparkListenerBlockManagerAdded): Option[Boolean] = { + Some(true) + } + + override def filterApplicationStart(event: SparkListenerApplicationStart): Option[Boolean] = { + Some(false) + } + } + + class TestEventFilter2 extends EventFilter { + override def filterApplicationEnd(event: SparkListenerApplicationEnd): Option[Boolean] = { + Some(true) + } + + override def filterEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Option[Boolean] = { + Some(true) + } + + override def filterBlockManagerAdded(event: SparkListenerBlockManagerAdded): Option[Boolean] = { + Some(false) + } + + override def filterApplicationStart(event: SparkListenerApplicationStart): Option[Boolean] = { + Some(false) + } + + override def filterNodeBlacklisted(event: SparkListenerNodeBlacklisted): Option[Boolean] = { + Some(true) + } + + override def filterNodeUnblacklisted(event: SparkListenerNodeUnblacklisted): Option[Boolean] = { + Some(false) + } + } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FilteredEventLogFileRewriterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FilteredEventLogFileRewriterSuite.scala index a554fbcc2031a..190103e8788a2 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FilteredEventLogFileRewriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FilteredEventLogFileRewriterSuite.scala @@ -25,6 +25,7 @@ import org.json4s.jackson.JsonMethods.{compact, render} import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.EventLogTestHelper.{TestEventFilter1, TestEventFilter2} import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{JsonProtocol, Utils} @@ -92,43 +93,3 @@ class FilteredEventLogFileRewriterSuite extends SparkFunSuite { compact(render(JsonProtocol.sparkEventToJson(event))) } } - -class TestEventFilter1 extends EventFilter { - override def filterApplicationEnd(event: SparkListenerApplicationEnd): Option[Boolean] = { - Some(true) - } - - override def filterBlockManagerAdded(event: SparkListenerBlockManagerAdded): Option[Boolean] = { - Some(true) - } - - override def filterApplicationStart(event: SparkListenerApplicationStart): Option[Boolean] = { - Some(false) - } -} - -class TestEventFilter2 extends EventFilter { - override def filterApplicationEnd(event: SparkListenerApplicationEnd): Option[Boolean] = { - Some(true) - } - - override def filterEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Option[Boolean] = { - Some(true) - } - - override def filterBlockManagerAdded(event: SparkListenerBlockManagerAdded): Option[Boolean] = { - Some(false) - } - - override def filterApplicationStart(event: SparkListenerApplicationStart): Option[Boolean] = { - Some(false) - } - - override def filterNodeBlacklisted(event: SparkListenerNodeBlacklisted): Option[Boolean] = { - Some(true) - } - - override def filterNodeUnblacklisted(event: SparkListenerNodeUnblacklisted): Option[Boolean] = { - Some(false) - } -} 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 4fd3f50338ff8..0e48992a6a59b 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 @@ -166,8 +166,9 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { override protected def mergeApplicationListing( reader: EventLogFileReader, lastSeen: Long, - enableSkipToEnd: Boolean): Unit = { - super.mergeApplicationListing(reader, lastSeen, enableSkipToEnd) + enableSkipToEnd: Boolean, + compactible: Option[Boolean]): Unit = { + super.mergeApplicationListing(reader, lastSeen, enableSkipToEnd, compactible) mergeApplicationListingCall += 1 } } @@ -1169,7 +1170,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { 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, None, false) + Some("attemptId"), 100, None, Some(false), false) var reader = EventLogFileReader(mockedFs, path) assert(reader.isDefined) assert(mockedProvider.shouldReloadLog(logInfo, reader.get)) @@ -1179,14 +1180,14 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { 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, None, false) + Some("attemptId"), 100, None, Some(false), 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, None, false) + Some("attemptId"), 200, None, Some(false), false) reader = EventLogFileReader(mockedFs, path) assert(reader.isDefined) assert(!mockedProvider.shouldReloadLog(logInfo, reader.get)) @@ -1294,11 +1295,11 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { val serializer = new KVStoreScalaSerializer() val logInfoWithIndexAsNone = LogInfo("dummy", 0, LogType.EventLogs, Some("appId"), - Some("attemptId"), 100, None, false) + Some("attemptId"), 100, None, None, false) assertSerDe(serializer, logInfoWithIndexAsNone) val logInfoWithIndex = LogInfo("dummy", 0, LogType.EventLogs, Some("appId"), - Some("attemptId"), 100, Some(3), false) + Some("attemptId"), 100, Some(3), Some(false), false) assertSerDe(serializer, logInfoWithIndex) } @@ -1315,11 +1316,11 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { val appInfo = new ApplicationAttemptInfo(None, new Date(1), new Date(1), new Date(1), 10, "spark", false, "dummy") val attemptInfoWithIndexAsNone = new AttemptInfoWrapper(appInfo, "dummyPath", 10, None, - None, None, None, None) + None, None, None, None, None) assertSerDe(serializer, attemptInfoWithIndexAsNone) val attemptInfoWithIndex = new AttemptInfoWrapper(appInfo, "dummyPath", 10, Some(1), - None, None, None, None) + Some(true), None, None, None, None) assertSerDe(serializer, attemptInfoWithIndex) } @@ -1360,7 +1361,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { when(reader.listEventLogFiles).thenReturn(logPaths.map { p => fs.getFileStatus(new Path(p)) }) val provider = new FsHistoryProvider(conf) - provider.rebuildAppStore(store, reader, 0) + provider.rebuildAppStore(store, reader, 0, Some(true)) // files being compacted are deleted assert(!fs.exists(new Path(logPath1))) @@ -1390,6 +1391,139 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { } } + class EligibilityCallTrackingFsHistoryProvider extends FsHistoryProvider(createTestConf()) { + override def mergeApplicationListing( + reader: EventLogFileReader, + scanTime: Long, + enableOptimizations: Boolean, + compactible: Option[Boolean]): Unit = { + super.mergeApplicationListing(reader, scanTime, enableOptimizations, compactible) + } + + var checkEligibilityForCompactionCall = 0 + override protected def checkEligibilityForCompaction( + info: LogInfo, + reader: EventLogFileReader): Option[Boolean] = { + val ret = super.checkEligibilityForCompaction(info, reader) + checkEligibilityForCompactionCall += 1 + ret + } + } + + private def createLogInfo(reader: EventLogFileReader, appId: String): LogInfo = { + LogInfo(reader.rootPath.toString, 0L, LogType.EventLogs, + Some(appId), None, reader.fileSizeForLastIndex, reader.lastIndex, + None, reader.completed) + } + + private def assertCompactibleInLogInfo( + provider: FsHistoryProvider, + reader: EventLogFileReader, + expected: Option[Boolean]): Unit = { + val logInfo = provider.listing.read(classOf[LogInfo], reader.rootPath.toString) + assert(expected === logInfo.compactible) + } + + test("update eligibility for compaction - rolling event log") { + def mockReaderForRollingEventLog( + fs: FileSystem, + rootPath: Path, + logPaths: Seq[Path]): EventLogFileReader = { + require(logPaths.nonEmpty) + val reader = mock(classOf[EventLogFileReader]) + val statuses = logPaths.map { log => fs.getFileStatus(log) } + when(reader.listEventLogFiles).thenReturn(statuses) + when(reader.lastIndex).thenReturn(Some(1L)) + when(reader.rootPath).thenReturn(rootPath) + when(reader.fileSizeForLastIndex).thenReturn(statuses.last.getLen) + when(reader.completed).thenReturn(false) + reader + } + + // To simplify the test, this test directly calls mergeApplicationListing() instead of going + // through checkForLogs(). + withTempDir { dir => + val provider = new EligibilityCallTrackingFsHistoryProvider + + val conf = createTestConf() + val hadoopConf = SparkHadoopUtil.newConfiguration(conf) + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + val logPath1 = EventLogTestHelper.writeEventLogFile(conf, hadoopConf, dir, 1, Seq( + SparkListenerApplicationStart("app", Some("app"), 0, "user", None))) + + val reader1 = mockReaderForRollingEventLog(fs, new Path(dir.getCanonicalPath), + Seq(new Path(logPath1))) + + provider.listing.write(createLogInfo(reader1, "app")) + provider.getOrUpdateCompactible(reader1) + + assert(1 === provider.checkEligibilityForCompactionCall) + // All events in log file are expected to be filtered in - so 'compactible' should be true, + // but we don't decide if there's only one file given the file might be updated later. + assertCompactibleInLogInfo(provider, reader1, None) + + // 10 of 11 events will be filtered out which meets the condition of compaction + val logPath2 = EventLogTestHelper.writeEventLogFile(conf, hadoopConf, dir, 2, + (1 to 5).flatMap { idx => + Seq(SparkListenerJobStart(idx, 1L, Seq.empty, null), + SparkListenerJobEnd(idx, 1L, JobSucceeded)) + } ++ Seq(SparkListenerUnpersistRDD(1))) + + val reader2 = mockReaderForRollingEventLog(fs, new Path(dir.getCanonicalPath), + Seq(new Path(logPath1), new Path(logPath2))) + + provider.listing.write(createLogInfo(reader2, "app")) + provider.checkEligibilityForCompactionCall = 0 + provider.getOrUpdateCompactible(reader2) + + assert(1 === provider.checkEligibilityForCompactionCall) + // All events in log file are expected to be filtered in - so 'compactible' should be false. + assertCompactibleInLogInfo(provider, reader2, Some(false)) + + val logPath3 = EventLogTestHelper.writeEventLogFile(conf, hadoopConf, dir, 3, Seq( + SparkListenerApplicationEnd(4L))) + + val reader3 = mockReaderForRollingEventLog(fs, new Path(dir.getCanonicalPath), + Seq(new Path(logPath2), new Path(logPath3))) + + provider.listing.write(createLogInfo(reader3, "app")) + provider.checkEligibilityForCompactionCall = 0 + provider.getOrUpdateCompactible(reader3) + + assert(1 === provider.checkEligibilityForCompactionCall) + // All events in log file are expected to be filtered out - so 'compactible' should be true. + assertCompactibleInLogInfo(provider, reader3, Some(true)) + } + } + + test("update eligibility for compaction - single event log") { + // To simplify the test, this test directly calls mergeApplicationListing() instead of going + // through checkForLogs(). + withTempDir { dir => + val provider = new EligibilityCallTrackingFsHistoryProvider + + val conf = createTestConf() + val hadoopConf = SparkHadoopUtil.newConfiguration(conf) + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + val singleLogFile = newLogFile("new1", None, inProgress = false) + writeFile(singleLogFile, None, + SparkListenerApplicationStart(singleLogFile.getName(), Some("new-app-complete"), 1L, "test", + None), + SparkListenerApplicationEnd(5L) + ) + + val readerForSingleLog = EventLogFileReader(fs, new Path(singleLogFile.getCanonicalPath)).get + provider.listing.write(createLogInfo(readerForSingleLog, "new1")) + + provider.getOrUpdateCompactible(readerForSingleLog) + + assert(1 === provider.checkEligibilityForCompactionCall) + assertCompactibleInLogInfo(provider, readerForSingleLog, Some(false)) + } + } + /** * Asks the provider to check for logs and calls a function to perform checks on the updated * app list. Example: diff --git a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala index 12e456f97ab20..37a35744ada5e 100644 --- a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala +++ b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala @@ -21,9 +21,9 @@ import java.util.Properties import scala.collection.immutable.Map -import org.apache.spark.{AccumulatorSuite, SparkContext} +import org.apache.spark.{AccumulatorSuite, SparkContext, Success, TaskState} import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} -import org.apache.spark.scheduler.{SparkListenerExecutorAdded, SparkListenerExecutorMetricsUpdate, SparkListenerExecutorRemoved, SparkListenerStageCompleted, SparkListenerStageSubmitted, SparkListenerTaskStart, StageInfo, TaskInfo, TaskLocality} +import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorAdded, SparkListenerExecutorMetricsUpdate, SparkListenerExecutorRemoved, SparkListenerJobStart, SparkListenerStageCompleted, SparkListenerStageSubmitted, SparkListenerTaskEnd, SparkListenerTaskStart, StageInfo, TaskInfo, TaskLocality} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage.{RDDInfo, StorageLevel} @@ -137,6 +137,53 @@ object ListenerEventsTestHelper { SparkListenerExecutorMetricsUpdate(executorId.toString, accum, executorUpdates) } + case class JobInfo( + stageIds: Seq[Int], + stageToTaskIds: Map[Int, Seq[Long]], + stageToRddIds: Map[Int, Seq[Int]]) + + def pushJobEventsWithoutJobEnd( + listener: SparkListener, + jobId: Int, + jobProps: Properties, + execIds: Array[String], + time: Long): JobInfo = { + // Start a job with 1 stages / 4 tasks each + val rddsForStage = createRdds(2) + val stage = createStage(rddsForStage, Nil) + + listener.onJobStart(SparkListenerJobStart(jobId, time, Seq(stage), jobProps)) + + // Submit stage + stage.submissionTime = Some(time) + listener.onStageSubmitted(SparkListenerStageSubmitted(stage, jobProps)) + + // Start tasks from stage + val s1Tasks = createTasks(4, execIds, time) + s1Tasks.foreach { task => + listener.onTaskStart(SparkListenerTaskStart(stage.stageId, + stage.attemptNumber(), task)) + } + + // Succeed all tasks in stage. + val s1Metrics = TaskMetrics.empty + s1Metrics.setExecutorCpuTime(2L) + s1Metrics.setExecutorRunTime(4L) + + s1Tasks.foreach { task => + task.markFinished(TaskState.FINISHED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, + "taskType", Success, task, new ExecutorMetrics, s1Metrics)) + } + + // End stage. + stage.completionTime = Some(time) + listener.onStageCompleted(SparkListenerStageCompleted(stage)) + + JobInfo(Seq(stage.stageId), Map(stage.stageId -> s1Tasks.map(_.taskId)), + Map(stage.stageId -> rddsForStage.map(_.id))) + } + private def nextTaskId(): Long = { taskIdTracker += 1 taskIdTracker diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala index 0c913e3a87476..7d86766f1c9ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.execution.history -import org.apache.spark.{SparkFunSuite, Success, TaskState} -import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} -import org.apache.spark.scheduler.{JobSucceeded, SparkListenerApplicationStart, SparkListenerJobEnd, SparkListenerJobStart, SparkListenerLogStart, SparkListenerStageCompleted, SparkListenerStageSubmitted, SparkListenerTaskEnd, SparkListenerTaskStart} +import java.util.Properties + +import org.apache.spark.SparkFunSuite +import org.apache.spark.scheduler._ import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart} import org.apache.spark.status.ListenerEventsTestHelper @@ -62,18 +63,23 @@ class SQLEventFilterBuilderSuite extends SparkFunSuite { time += 1 // job 1, 2: coupled with SQL execution 1, finished - val jobInfoForJob1 = pushJobEventsWithoutJobEnd(listener, 1, execIds, Some("1"), time) + val jobProp = createJobProps() + val jobPropWithSqlExecution = new Properties(jobProp) + jobPropWithSqlExecution.setProperty(SQLExecution.EXECUTION_ID_KEY, "1") + val jobInfoForJob1 = pushJobEventsWithoutJobEnd(listener, 1, jobPropWithSqlExecution, + execIds, time) listener.onJobEnd(SparkListenerJobEnd(1, time, JobSucceeded)) - val jobInfoForJob2 = pushJobEventsWithoutJobEnd(listener, 2, execIds, Some("1"), time) + val jobInfoForJob2 = pushJobEventsWithoutJobEnd(listener, 2, jobPropWithSqlExecution, + execIds, time) listener.onJobEnd(SparkListenerJobEnd(2, time, JobSucceeded)) // job 3: not coupled with SQL execution 1, finished - pushJobEventsWithoutJobEnd(listener, 3, execIds, None, time) + pushJobEventsWithoutJobEnd(listener, 3, jobProp, execIds, time) listener.onJobEnd(SparkListenerJobEnd(3, time, JobSucceeded)) // job 4: not coupled with SQL execution 1, not finished - pushJobEventsWithoutJobEnd(listener, 4, execIds, None, time) + pushJobEventsWithoutJobEnd(listener, 4, jobProp, execIds, time) listener.onJobEnd(SparkListenerJobEnd(4, time, JobSucceeded)) assert(listener.liveExecutionToJobs.keys === Set(1)) @@ -108,54 +114,4 @@ class SQLEventFilterBuilderSuite extends SparkFunSuite { assert(listener.stageToTasks.isEmpty) assert(listener.stageToRDDs.isEmpty) } - - case class JobInfo( - stageIds: Seq[Int], - stageToTaskIds: Map[Int, Seq[Long]], - stageToRddIds: Map[Int, Seq[Int]]) - - private def pushJobEventsWithoutJobEnd( - listener: SQLEventFilterBuilder, - jobId: Int, - execIds: Array[String], - sqlExecId: Option[String], - time: Long): JobInfo = { - // Start a job with 1 stages / 4 tasks each - val rddsForStage = createRdds(2) - val stage = createStage(rddsForStage, Nil) - - val jobProps = createJobProps() - sqlExecId.foreach { id => jobProps.setProperty(SQLExecution.EXECUTION_ID_KEY, id) } - - listener.onJobStart(SparkListenerJobStart(jobId, time, Seq(stage), jobProps)) - - // Submit stage - stage.submissionTime = Some(time) - listener.onStageSubmitted(SparkListenerStageSubmitted(stage, jobProps)) - - // Start tasks from stage - val s1Tasks = ListenerEventsTestHelper.createTasks(4, execIds, time) - s1Tasks.foreach { task => - listener.onTaskStart(SparkListenerTaskStart(stage.stageId, - stage.attemptNumber(), task)) - } - - // Succeed all tasks in stage. - val s1Metrics = TaskMetrics.empty - s1Metrics.setExecutorCpuTime(2L) - s1Metrics.setExecutorRunTime(4L) - - s1Tasks.foreach { task => - task.markFinished(TaskState.FINISHED, time) - listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, - "taskType", Success, task, new ExecutorMetrics, s1Metrics)) - } - - // End stage. - stage.completionTime = Some(time) - listener.onStageCompleted(SparkListenerStageCompleted(stage)) - - JobInfo(Seq(stage.stageId), Map(stage.stageId -> s1Tasks.map(_.taskId)), - Map(stage.stageId -> rddsForStage.map(_.id))) - } } From 82e268a068b3efc0b4f75dfde362a3045f6a8fd9 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 27 Nov 2019 13:44:23 +0900 Subject: [PATCH 18/28] Clean a bit --- .../deploy/history/FsHistoryProviderSuite.scala | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) 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 0e48992a6a59b..e2ec8778ce5b3 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 @@ -1392,14 +1392,6 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { } class EligibilityCallTrackingFsHistoryProvider extends FsHistoryProvider(createTestConf()) { - override def mergeApplicationListing( - reader: EventLogFileReader, - scanTime: Long, - enableOptimizations: Boolean, - compactible: Option[Boolean]): Unit = { - super.mergeApplicationListing(reader, scanTime, enableOptimizations, compactible) - } - var checkEligibilityForCompactionCall = 0 override protected def checkEligibilityForCompaction( info: LogInfo, @@ -1492,14 +1484,13 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { provider.getOrUpdateCompactible(reader3) assert(1 === provider.checkEligibilityForCompactionCall) - // All events in log file are expected to be filtered out - so 'compactible' should be true. + // Most of events in log file are expected to be filtered out - so 'compactible' should + // be true. assertCompactibleInLogInfo(provider, reader3, Some(true)) } } test("update eligibility for compaction - single event log") { - // To simplify the test, this test directly calls mergeApplicationListing() instead of going - // through checkForLogs(). withTempDir { dir => val provider = new EligibilityCallTrackingFsHistoryProvider From e5d925025a606cbb5c365303149272900f255e33 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 28 Nov 2019 10:11:13 +0900 Subject: [PATCH 19/28] Remove invalid comments, strengthen doc --- .../history/FilteredEventLogFileRewriterSuite.scala | 9 ++------- .../spark/deploy/history/FsHistoryProviderSuite.scala | 2 -- docs/configuration.md | 9 ++++++++- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FilteredEventLogFileRewriterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FilteredEventLogFileRewriterSuite.scala index 190103e8788a2..96883a7647b4e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FilteredEventLogFileRewriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FilteredEventLogFileRewriterSuite.scala @@ -21,19 +21,18 @@ import scala.collection.mutable import scala.io.{Codec, Source} import org.apache.hadoop.fs.Path -import org.json4s.jackson.JsonMethods.{compact, render} import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.EventLogTestHelper.{TestEventFilter1, TestEventFilter2} import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{JsonProtocol, Utils} +import org.apache.spark.util.Utils class FilteredEventLogFileRewriterSuite extends SparkFunSuite { test("rewrite files with test filters") { def writeEventToWriter(writer: EventLogFileWriter, event: SparkListenerEvent): String = { - val line = convertEvent(event) + val line = EventLogTestHelper.convertEvent(event) writer.writeEvent(line, flushLogger = true) line } @@ -88,8 +87,4 @@ class FilteredEventLogFileRewriterSuite extends SparkFunSuite { } } } - - private def convertEvent(event: SparkListenerEvent): String = { - compact(render(JsonProtocol.sparkEventToJson(event))) - } } 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 e2ec8778ce5b3..f2ea999356630 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 @@ -1432,8 +1432,6 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { reader } - // To simplify the test, this test directly calls mergeApplicationListing() instead of going - // through checkForLogs(). withTempDir { dir => val provider = new EligibilityCallTrackingFsHistoryProvider diff --git a/docs/configuration.md b/docs/configuration.md index 9cc8696b312db..41b8f513acde5 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1031,7 +1031,14 @@ Apart from these, the following properties are also available, and may be useful By default, all event log files will be retained. Please set the configuration and spark.eventLog.rolling.maxFileSize accordingly if you want to control the overall size of event log files. The event log files older than these retained - files will be compacted into single file and deleted afterwards. + files will be compacted into single file and deleted afterwards.
+ NOTE 1: Compaction will happen in Spark History Server, which means the same value + will be applied across applications which are being loaded in Spark History Server, + as well as compaction and cleanup would require running Spark History Server.
+ NOTE 2: Spark History Server may not compact the old event log files if it figures + out compaction on event log for such application won't reduce the size at expected + rate threshold. For streaming query (including Structured Streaming) we normally + expect compaction will run, but for batch query compaction won't run in most cases. From b53aaf216abc0bdb71043da9ab67de772a897464 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 9 Dec 2019 21:37:52 +0900 Subject: [PATCH 20/28] Address review comments --- .../history/BasicEventFilterBuilder.scala | 85 +++++------ .../spark/deploy/history/EventFilter.scala | 135 +++--------------- .../history/EventLogFileCompactor.scala | 39 +++-- .../EventLogFilterRateCalculator.scala | 45 +++--- .../history/BasicEventFilterSuite.scala | 63 ++++---- .../deploy/history/EventLogTestHelper.scala | 45 ++---- .../history/SQLEventFilterBuilder.scala | 84 +++-------- .../SQLLiveEntitiesEventFilterSuite.scala | 52 ++++--- 8 files changed, 171 insertions(+), 377 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala index 90dbcab6492fb..14d8a2bba9d50 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala @@ -98,48 +98,38 @@ private[spark] abstract class JobEventFilter( logDebug(s"tasks in stages : ${stageToTasks.values.flatten}") logDebug(s"RDDs in stages : ${stageToRDDs.values.flatten}") - override def filterStageCompleted(event: SparkListenerStageCompleted): Option[Boolean] = { - Some(stageToTasks.contains(event.stageInfo.stageId)) - } + protected val acceptFnForJobEvents: PartialFunction[SparkListenerEvent, Boolean] = { + case e: SparkListenerStageCompleted => + stageToTasks.contains(e.stageInfo.stageId) - override def filterStageSubmitted(event: SparkListenerStageSubmitted): Option[Boolean] = { - Some(stageToTasks.contains(event.stageInfo.stageId)) - } + case e: SparkListenerStageSubmitted => + stageToTasks.contains(e.stageInfo.stageId) - override def filterTaskStart(event: SparkListenerTaskStart): Option[Boolean] = { - Some(liveTasks.contains(event.taskInfo.taskId)) - } + case e: SparkListenerTaskStart => + liveTasks.contains(e.taskInfo.taskId) - override def filterTaskGettingResult(event: SparkListenerTaskGettingResult): Option[Boolean] = { - Some(liveTasks.contains(event.taskInfo.taskId)) - } + case e: SparkListenerTaskGettingResult => + liveTasks.contains(e.taskInfo.taskId) - override def filterTaskEnd(event: SparkListenerTaskEnd): Option[Boolean] = { - Some(liveTasks.contains(event.taskInfo.taskId)) - } + case e: SparkListenerTaskEnd => + liveTasks.contains(e.taskInfo.taskId) - override def filterJobStart(event: SparkListenerJobStart): Option[Boolean] = { - Some(jobToStages.contains(event.jobId)) - } + case e: SparkListenerJobStart => + jobToStages.contains(e.jobId) - override def filterJobEnd(event: SparkListenerJobEnd): Option[Boolean] = { - Some(jobToStages.contains(event.jobId)) - } + case e: SparkListenerJobEnd => + jobToStages.contains(e.jobId) - override def filterUnpersistRDD(event: SparkListenerUnpersistRDD): Option[Boolean] = { - Some(liveRDDs.contains(event.rddId)) - } + case e: SparkListenerUnpersistRDD => + liveRDDs.contains(e.rddId) - override def filterExecutorMetricsUpdate( - event: SparkListenerExecutorMetricsUpdate): Option[Boolean] = { - Some(event.accumUpdates.exists { case (_, stageId, _, _) => - stageToTasks.contains(stageId) - }) - } + case e: SparkListenerExecutorMetricsUpdate => + e.accumUpdates.exists { case (_, stageId, _, _) => + stageToTasks.contains(stageId) + } - override def filterSpeculativeTaskSubmitted( - event: SparkListenerSpeculativeTaskSubmitted): Option[Boolean] = { - Some(stageToTasks.contains(event.stageId)) + case e: SparkListenerSpeculativeTaskSubmitted => + stageToTasks.contains(e.stageId) } } @@ -157,27 +147,16 @@ private[spark] class BasicEventFilter( logDebug(s"live executors : $liveExecutors") - override def filterExecutorAdded(event: SparkListenerExecutorAdded): Option[Boolean] = { - Some(liveExecutors.contains(event.executorId)) - } - - override def filterExecutorRemoved(event: SparkListenerExecutorRemoved): Option[Boolean] = { - Some(liveExecutors.contains(event.executorId)) - } - - override def filterExecutorBlacklisted( - event: SparkListenerExecutorBlacklisted): Option[Boolean] = { - Some(liveExecutors.contains(event.executorId)) - } - - override def filterExecutorUnblacklisted( - event: SparkListenerExecutorUnblacklisted): Option[Boolean] = { - Some(liveExecutors.contains(event.executorId)) - } + override def accept(event: SparkListenerEvent): Option[Boolean] = { + val fn: PartialFunction[SparkListenerEvent, Boolean] = { + case e: SparkListenerExecutorAdded => liveExecutors.contains(e.executorId) + case e: SparkListenerExecutorRemoved => liveExecutors.contains(e.executorId) + case e: SparkListenerExecutorBlacklisted => liveExecutors.contains(e.executorId) + case e: SparkListenerExecutorUnblacklisted => liveExecutors.contains(e.executorId) + case e: SparkListenerStageExecutorMetrics => liveExecutors.contains(e.execId) + } - override def filterStageExecutorMetrics( - event: SparkListenerStageExecutorMetrics): Option[Boolean] = { - Some(liveExecutors.contains(event.execId)) + fn.orElse(acceptFnForJobEvents).lift.apply(event) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala index 72137a228bfd4..fbc238cb49137 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala @@ -62,121 +62,34 @@ object EventFilterBuilder { } /** - * [[EventFilter]] decides whether the given event should be filtered in, or filtered out when - * compacting event log files. - * - * The meaning of return values of each filterXXX method are following: - * - Some(true): Filter in this event. - * - Some(false): Filter out this event. - * - None: Don't mind about this event. No problem even other filters decide to filter out. + * [[EventFilter]] decides whether the given event should be accepted or rejected. * * Please refer [[FilteredEventLogFileRewriter]] for more details on how the filter will be used. */ private[spark] trait EventFilter { - def filterStageCompleted(event: SparkListenerStageCompleted): Option[Boolean] = None - - def filterStageSubmitted(event: SparkListenerStageSubmitted): Option[Boolean] = None - - def filterTaskStart(event: SparkListenerTaskStart): Option[Boolean] = None - - def filterTaskGettingResult(event: SparkListenerTaskGettingResult): Option[Boolean] = None - - def filterTaskEnd(event: SparkListenerTaskEnd): Option[Boolean] = None - - def filterJobStart(event: SparkListenerJobStart): Option[Boolean] = None - - def filterJobEnd(event: SparkListenerJobEnd): Option[Boolean] = None - - def filterEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Option[Boolean] = None - - def filterBlockManagerAdded(event: SparkListenerBlockManagerAdded): Option[Boolean] = None - - def filterBlockManagerRemoved(event: SparkListenerBlockManagerRemoved): Option[Boolean] = None - - def filterUnpersistRDD(event: SparkListenerUnpersistRDD): Option[Boolean] = None - - def filterApplicationStart(event: SparkListenerApplicationStart): Option[Boolean] = None - - def filterApplicationEnd(event: SparkListenerApplicationEnd): Option[Boolean] = None - - def filterExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Option[Boolean] = None - - def filterStageExecutorMetrics(event: SparkListenerStageExecutorMetrics): Option[Boolean] = None - - def filterExecutorAdded(event: SparkListenerExecutorAdded): Option[Boolean] = None - - def filterExecutorRemoved(event: SparkListenerExecutorRemoved): Option[Boolean] = None - - def filterExecutorBlacklisted(event: SparkListenerExecutorBlacklisted): Option[Boolean] = None - - def filterExecutorBlacklistedForStage( - event: SparkListenerExecutorBlacklistedForStage): Option[Boolean] = None - - def filterNodeBlacklistedForStage( - event: SparkListenerNodeBlacklistedForStage): Option[Boolean] = None - - def filterExecutorUnblacklisted(event: SparkListenerExecutorUnblacklisted): Option[Boolean] = None - - def filterNodeBlacklisted(event: SparkListenerNodeBlacklisted): Option[Boolean] = None - - def filterNodeUnblacklisted(event: SparkListenerNodeUnblacklisted): Option[Boolean] = None - - def filterBlockUpdated(event: SparkListenerBlockUpdated): Option[Boolean] = None - - def filterSpeculativeTaskSubmitted( - event: SparkListenerSpeculativeTaskSubmitted): Option[Boolean] = None - - def filterOtherEvent(event: SparkListenerEvent): Option[Boolean] = None + /** + * Classify whether the event is accepted or rejected by this filter. + * + * Note that the method signature requires to return Option[Boolean] instead of Boolean: + * if the filter is able to decide with event whether it should be accepted or rejected, + * it can return either Some(true) or Some(false). Otherwise it should return None. + */ + def accept(event: SparkListenerEvent): Option[Boolean] } -object EventFilter { +object EventFilter extends Logging { def checkFilters(filters: Seq[EventFilter], event: SparkListenerEvent): Boolean = { - val results = filters.flatMap(filter => applyFilter(filter, event)) + val results = filters.flatMap(_.accept(event)) results.isEmpty || results.forall(_ == true) } - private def applyFilter(filter: EventFilter, event: SparkListenerEvent): Option[Boolean] = { - // This pattern match should have same list of event types, but it would be safe even if - // it's out of sync, once filter doesn't mark events to filter out for unknown event types. - event match { - case event: SparkListenerStageSubmitted => filter.filterStageSubmitted(event) - case event: SparkListenerStageCompleted => filter.filterStageCompleted(event) - case event: SparkListenerJobStart => filter.filterJobStart(event) - case event: SparkListenerJobEnd => filter.filterJobEnd(event) - case event: SparkListenerTaskStart => filter.filterTaskStart(event) - case event: SparkListenerTaskGettingResult => filter.filterTaskGettingResult(event) - case event: SparkListenerTaskEnd => filter.filterTaskEnd(event) - case event: SparkListenerEnvironmentUpdate => filter.filterEnvironmentUpdate(event) - case event: SparkListenerBlockManagerAdded => filter.filterBlockManagerAdded(event) - case event: SparkListenerBlockManagerRemoved => filter.filterBlockManagerRemoved(event) - case event: SparkListenerUnpersistRDD => filter.filterUnpersistRDD(event) - case event: SparkListenerApplicationStart => filter.filterApplicationStart(event) - case event: SparkListenerApplicationEnd => filter.filterApplicationEnd(event) - case event: SparkListenerExecutorMetricsUpdate => filter.filterExecutorMetricsUpdate(event) - case event: SparkListenerStageExecutorMetrics => filter.filterStageExecutorMetrics(event) - case event: SparkListenerExecutorAdded => filter.filterExecutorAdded(event) - case event: SparkListenerExecutorRemoved => filter.filterExecutorRemoved(event) - case event: SparkListenerExecutorBlacklistedForStage => - filter.filterExecutorBlacklistedForStage(event) - case event: SparkListenerNodeBlacklistedForStage => - filter.filterNodeBlacklistedForStage(event) - case event: SparkListenerExecutorBlacklisted => filter.filterExecutorBlacklisted(event) - case event: SparkListenerExecutorUnblacklisted => filter.filterExecutorUnblacklisted(event) - case event: SparkListenerNodeBlacklisted => filter.filterNodeBlacklisted(event) - case event: SparkListenerNodeUnblacklisted => filter.filterNodeUnblacklisted(event) - case event: SparkListenerBlockUpdated => filter.filterBlockUpdated(event) - case event: SparkListenerSpeculativeTaskSubmitted => - filter.filterSpeculativeTaskSubmitted(event) - case _ => filter.filterOtherEvent(event) - } - } -} - -trait EventFilterApplier extends Logging { - val fs: FileSystem - val filters: Seq[EventFilter] - - def applyFilter(path: Path): Unit = { + def applyFilterToFile( + fs: FileSystem, + filters: Seq[EventFilter], + path: Path)( + fnAccepted: (String, SparkListenerEvent) => Unit)( + fnRejected: (String, SparkListenerEvent) => Unit)( + fnUnidentified: String => Unit): Unit = { Utils.tryWithResource(EventLogFileReader.openEventLog(path, fs)) { in => val lines = Source.fromInputStream(in)(Codec.UTF8).getLines() @@ -197,15 +110,15 @@ trait EventFilterApplier extends Logging { // ignore any exception occurred from unidentified json // just skip handling and write the line case NonFatal(_) => - handleUnidentifiedLine(currentLine) + fnUnidentified(currentLine) None } event.foreach { e => if (EventFilter.checkFilters(filters, e)) { - handleFilteredInEvent(currentLine, e) + fnAccepted(currentLine, e) } else { - handleFilteredOutEvent(currentLine, e) + fnRejected(currentLine, e) } } } @@ -217,10 +130,4 @@ trait EventFilterApplier extends Logging { } } } - - protected def handleFilteredInEvent(line: String, event: SparkListenerEvent): Unit - - protected def handleFilteredOutEvent(line: String, event: SparkListenerEvent): Unit - - protected def handleUnidentifiedLine(line: String): Unit } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index 8a190bc4c3fb0..792f122c57c01 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -113,34 +113,29 @@ class EventLogFileCompactor( class FilteredEventLogFileRewriter( sparkConf: SparkConf, hadoopConf: Configuration, - override val fs: FileSystem, - override val filters: Seq[EventFilter]) extends EventFilterApplier { - - private var logWriter: Option[CompactedEventLogFileWriter] = None + fs: FileSystem, + filters: Seq[EventFilter]) { def rewrite(eventLogFiles: Seq[FileStatus]): String = { require(eventLogFiles.nonEmpty) val targetEventLogFilePath = eventLogFiles.last.getPath - logWriter = Some(new CompactedEventLogFileWriter(targetEventLogFilePath, "dummy", None, - targetEventLogFilePath.getParent.toUri, sparkConf, hadoopConf)) - - val writer = logWriter.get - writer.start() - eventLogFiles.foreach { file => applyFilter(file.getPath) } - writer.stop() - - writer.logPath - } - - override protected def handleFilteredInEvent(line: String, event: SparkListenerEvent): Unit = { - logWriter.foreach { writer => writer.writeEvent(line, flushLogger = true) } - } - - override protected def handleFilteredOutEvent(line: String, event: SparkListenerEvent): Unit = {} + val logWriter = new CompactedEventLogFileWriter(targetEventLogFilePath, "dummy", None, + targetEventLogFilePath.getParent.toUri, sparkConf, hadoopConf) + + logWriter.start() + eventLogFiles.foreach { file => + EventFilter.applyFilterToFile(fs, filters, file.getPath) { case (line, _) => + logWriter.writeEvent(line, flushLogger = true) + } { case (_, _) => + // no-op + } { line => + logWriter.writeEvent(line, flushLogger = true) + } + } + logWriter.stop() - override protected def handleUnidentifiedLine(line: String): Unit = { - logWriter.foreach { writer => writer.writeEvent(line, flushLogger = true) } + logWriter.logPath } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculator.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculator.scala index 9d1396641525f..db1ce3c0dcb86 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculator.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculator.scala @@ -22,12 +22,12 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.scheduler.SparkListenerEvent /** - * This class calculates the rate of events being filtered-in, via two phases reading: + * This class calculates the rate of events being accepted, via two phases reading: * * 1) Initialize available [[EventFilterBuilder]] instances, and replay the event log files with * builders, so that these builders can gather the information to create [[EventFilter]] instances. * 2) Initialize [[EventFilter]] instances from [[EventFilterBuilder]] instances, and replay the - * event log files with filters. Counts the number of events vs filtered-in events and calculate + * event log files with filters. Counts the number of events vs accepted events and calculate * the rate. */ class EventLogFilterRateCalculator(fs: FileSystem) { @@ -45,35 +45,24 @@ class EventLogFilterRateCalculator(fs: FileSystem) { calc.calculate(eventLogPaths) } - private class Calculator( - override val fs: FileSystem, - override val filters: Seq[EventFilter]) extends EventFilterApplier { - private var allEvents = 0L - private var filteredInEvents = 0L - + private class Calculator(fs: FileSystem, filters: Seq[EventFilter]) { def calculate(eventLogPaths: Seq[Path]): Double = { - clearValues() - eventLogPaths.foreach { path => applyFilter(path) } - filteredInEvents.toDouble / allEvents - } + var allEvents = 0L + var filteredInEvents = 0L - protected def handleFilteredInEvent(line: String, event: SparkListenerEvent): Unit = { - allEvents += 1 - filteredInEvents += 1 - } + eventLogPaths.foreach { path => + EventFilter.applyFilterToFile(fs, filters, path) { case (_, _) => + allEvents += 1 + filteredInEvents += 1 + } { case (_, _) => + allEvents += 1 + } { _ => + allEvents += 1 + filteredInEvents += 1 + } + } - protected def handleFilteredOutEvent(line: String, event: SparkListenerEvent): Unit = { - allEvents += 1 - } - - protected def handleUnidentifiedLine(line: String): Unit = { - allEvents += 1 - filteredInEvents += 1 - } - - private def clearValues(): Unit = { - allEvents = 0L - filteredInEvents = 0L + filteredInEvents.toDouble / allEvents } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala index 5e9b067a9607b..f71f4f1840f7b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala @@ -76,7 +76,7 @@ class BasicEventFilterSuite extends SparkFunSuite { val unpersistRDDEventsForJob2 = rddsForStage2.map { rdd => SparkListenerUnpersistRDD(rdd.id) } // job events for live job should be filtered in - assert(filter.filterJobStart(jobStartEventForJob2) === Some(true)) + assert(filter.accept(jobStartEventForJob2) === Some(true)) // stage events for live job should be filtered in assertFilterStageEvents( @@ -98,28 +98,24 @@ class BasicEventFilterSuite extends SparkFunSuite { val filter = new BasicEventFilter(Map.empty, Map.empty, Map.empty, liveExecutors) // events for dead executor should be filtered out - assert(filter.filterExecutorAdded(createExecutorAddedEvent(1)) === Some(false)) + assert(filter.accept(createExecutorAddedEvent(1)) === Some(false)) // though the name of event is stage executor metrics, AppStatusListener only deals with // live executors - assert(filter.filterStageExecutorMetrics( + assert(filter.accept( SparkListenerStageExecutorMetrics(1.toString, 0, 0, new ExecutorMetrics)) === Some(false)) - assert(filter.filterExecutorBlacklisted(SparkListenerExecutorBlacklisted(0, 1.toString, 1)) - === Some(false)) - assert(filter.filterExecutorUnblacklisted(SparkListenerExecutorUnblacklisted(0, 1.toString)) - === Some(false)) - assert(filter.filterExecutorRemoved(createExecutorRemovedEvent(1)) === Some(false)) + assert(filter.accept(SparkListenerExecutorBlacklisted(0, 1.toString, 1)) === Some(false)) + assert(filter.accept(SparkListenerExecutorUnblacklisted(0, 1.toString)) === Some(false)) + assert(filter.accept(createExecutorRemovedEvent(1)) === Some(false)) // events for live executor should be filtered in - assert(filter.filterExecutorAdded(createExecutorAddedEvent(2)) === Some(true)) - assert(filter.filterStageExecutorMetrics( + assert(filter.accept(createExecutorAddedEvent(2)) === Some(true)) + assert(filter.accept( SparkListenerStageExecutorMetrics(2.toString, 0, 0, new ExecutorMetrics)) === Some(true)) - assert(filter.filterExecutorBlacklisted(SparkListenerExecutorBlacklisted(0, 2.toString, 1)) - === Some(true)) - assert(filter.filterExecutorUnblacklisted(SparkListenerExecutorUnblacklisted(0, 2.toString)) - === Some(true)) - assert(filter.filterExecutorRemoved(createExecutorRemovedEvent(2)) === Some(true)) + assert(filter.accept(SparkListenerExecutorBlacklisted(0, 2.toString, 1)) === Some(true)) + assert(filter.accept(SparkListenerExecutorUnblacklisted(0, 2.toString)) === Some(true)) + assert(filter.accept(createExecutorRemovedEvent(2)) === Some(true)) } test("other events should be left to other filters") { @@ -129,18 +125,17 @@ class BasicEventFilterSuite extends SparkFunSuite { val filter = new BasicEventFilter(Map.empty, Map.empty, Map.empty, Set.empty) - assertNone(filter.filterEnvironmentUpdate(SparkListenerEnvironmentUpdate(Map.empty))) - assertNone(filter.filterApplicationStart( - SparkListenerApplicationStart("1", Some("1"), 0, "user", None))) - assertNone(filter.filterApplicationEnd(SparkListenerApplicationEnd(1))) + assertNone(filter.accept(SparkListenerEnvironmentUpdate(Map.empty))) + assertNone(filter.accept(SparkListenerApplicationStart("1", Some("1"), 0, "user", None))) + assertNone(filter.accept(SparkListenerApplicationEnd(1))) val bmId = BlockManagerId("1", "host1", 1) - assertNone(filter.filterBlockManagerAdded(SparkListenerBlockManagerAdded(0, bmId, 1))) - assertNone(filter.filterBlockManagerRemoved(SparkListenerBlockManagerRemoved(1, bmId))) - assertNone(filter.filterBlockUpdated(SparkListenerBlockUpdated( + assertNone(filter.accept(SparkListenerBlockManagerAdded(0, bmId, 1))) + assertNone(filter.accept(SparkListenerBlockManagerRemoved(1, bmId))) + assertNone(filter.accept(SparkListenerBlockUpdated( storage.BlockUpdatedInfo(bmId, RDDBlockId(1, 1), StorageLevel.DISK_ONLY, 0, 10)))) - assertNone(filter.filterNodeBlacklisted(SparkListenerNodeBlacklisted(0, "host1", 1))) - assertNone(filter.filterNodeUnblacklisted(SparkListenerNodeUnblacklisted(0, "host1"))) - assertNone(filter.filterOtherEvent(SparkListenerLogStart("testVersion"))) + assertNone(filter.accept(SparkListenerNodeBlacklisted(0, "host1", 1))) + assertNone(filter.accept(SparkListenerNodeUnblacklisted(0, "host1"))) + assertNone(filter.accept(SparkListenerLogStart("testVersion"))) } private def assertFilterJobEvents( @@ -148,8 +143,8 @@ class BasicEventFilterSuite extends SparkFunSuite { jobStart: SparkListenerJobStart, jobEnd: SparkListenerJobEnd, expectedVal: Option[Boolean]): Unit = { - assert(filter.filterJobStart(jobStart) === expectedVal) - assert(filter.filterJobEnd(jobEnd) === expectedVal) + assert(filter.accept(jobStart) === expectedVal) + assert(filter.accept(jobEnd) === expectedVal) } private def assertFilterStageEvents( @@ -159,12 +154,12 @@ class BasicEventFilterSuite extends SparkFunSuite { unpersistRDDs: Seq[SparkListenerUnpersistRDD], taskSpeculativeSubmitted: SparkListenerSpeculativeTaskSubmitted, expectedVal: Option[Boolean]): Unit = { - assert(filter.filterStageSubmitted(stageSubmitted) === expectedVal) - assert(filter.filterStageCompleted(stageCompleted) === expectedVal) + assert(filter.accept(stageSubmitted) === expectedVal) + assert(filter.accept(stageCompleted) === expectedVal) unpersistRDDs.foreach { event => - assert(filter.filterUnpersistRDD(event) === expectedVal) + assert(filter.accept(event) === expectedVal) } - assert(filter.filterSpeculativeTaskSubmitted(taskSpeculativeSubmitted) === expectedVal) + assert(filter.accept(taskSpeculativeSubmitted) === expectedVal) } private def assertFilterTaskEvents( @@ -174,14 +169,14 @@ class BasicEventFilterSuite extends SparkFunSuite { expectedVal: Option[Boolean]): Unit = { taskInfos.foreach { task => val taskStartEvent = SparkListenerTaskStart(stageInfo.stageId, 0, task) - assert(filter.filterTaskStart(taskStartEvent) === expectedVal) + assert(filter.accept(taskStartEvent) === expectedVal) val taskGettingResultEvent = SparkListenerTaskGettingResult(task) - assert(filter.filterTaskGettingResult(taskGettingResultEvent) === expectedVal) + assert(filter.accept(taskGettingResultEvent) === expectedVal) val taskEndEvent = SparkListenerTaskEnd(stageInfo.stageId, 0, "taskType", Success, task, new ExecutorMetrics, null) - assert(filter.filterTaskEnd(taskEndEvent) === expectedVal) + assert(filter.accept(taskEndEvent) === expectedVal) } } } 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 0b16121f20249..a119abf9b4335 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 @@ -82,42 +82,23 @@ object EventLogTestHelper { } class TestEventFilter1 extends EventFilter { - override def filterApplicationEnd(event: SparkListenerApplicationEnd): Option[Boolean] = { - Some(true) - } - - override def filterBlockManagerAdded(event: SparkListenerBlockManagerAdded): Option[Boolean] = { - Some(true) - } - - override def filterApplicationStart(event: SparkListenerApplicationStart): Option[Boolean] = { - Some(false) + override def accept(event: SparkListenerEvent): Option[Boolean] = event match { + case _: SparkListenerApplicationEnd => Some(true) + case _: SparkListenerBlockManagerAdded => Some(true) + case _: SparkListenerApplicationStart => Some(false) + case _ => None } } class TestEventFilter2 extends EventFilter { - override def filterApplicationEnd(event: SparkListenerApplicationEnd): Option[Boolean] = { - Some(true) - } - - override def filterEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Option[Boolean] = { - Some(true) - } - - override def filterBlockManagerAdded(event: SparkListenerBlockManagerAdded): Option[Boolean] = { - Some(false) - } - - override def filterApplicationStart(event: SparkListenerApplicationStart): Option[Boolean] = { - Some(false) - } - - override def filterNodeBlacklisted(event: SparkListenerNodeBlacklisted): Option[Boolean] = { - Some(true) - } - - override def filterNodeUnblacklisted(event: SparkListenerNodeUnblacklisted): Option[Boolean] = { - Some(false) + override def accept(event: SparkListenerEvent): Option[Boolean] = event match { + case _: SparkListenerApplicationEnd => Some(true) + case _: SparkListenerEnvironmentUpdate => Some(true) + case _: SparkListenerNodeBlacklisted => Some(true) + case _: SparkListenerBlockManagerAdded => Some(false) + case _: SparkListenerApplicationStart => Some(false) + case _: SparkListenerNodeUnblacklisted => Some(false) + case _ => None } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala index 82e15bd80d122..8cedd4d7be31d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala @@ -126,74 +126,24 @@ private[spark] class SQLLiveEntitiesEventFilter( logDebug(s"live executions : ${liveExecutionToJobs.keySet}") logDebug(s"jobs in live executions : ${liveExecutionToJobs.values.flatten}") - override def filterStageCompleted(event: SparkListenerStageCompleted): Option[Boolean] = { - trueOrNone(super.filterStageCompleted(event)) - } - - override def filterStageSubmitted(event: SparkListenerStageSubmitted): Option[Boolean] = { - trueOrNone(super.filterStageSubmitted(event)) - } - - override def filterTaskStart(event: SparkListenerTaskStart): Option[Boolean] = { - trueOrNone(super.filterTaskStart(event)) - } - - override def filterTaskGettingResult(event: SparkListenerTaskGettingResult): Option[Boolean] = { - trueOrNone(super.filterTaskGettingResult(event)) - } - - override def filterTaskEnd(event: SparkListenerTaskEnd): Option[Boolean] = { - trueOrNone(super.filterTaskEnd(event)) - } - - override def filterJobStart(event: SparkListenerJobStart): Option[Boolean] = { - trueOrNone(super.filterJobStart(event)) - } - - override def filterJobEnd(event: SparkListenerJobEnd): Option[Boolean] = { - trueOrNone(super.filterJobEnd(event)) - } - - override def filterUnpersistRDD(event: SparkListenerUnpersistRDD): Option[Boolean] = { - trueOrNone(super.filterUnpersistRDD(event)) - } - - override def filterExecutorMetricsUpdate( - event: SparkListenerExecutorMetricsUpdate): Option[Boolean] = { - trueOrNone(super.filterExecutorMetricsUpdate(event)) - } - - override def filterSpeculativeTaskSubmitted( - event: SparkListenerSpeculativeTaskSubmitted): Option[Boolean] = { - trueOrNone(super.filterSpeculativeTaskSubmitted(event)) - } - - override def filterOtherEvent(event: SparkListenerEvent): Option[Boolean] = event match { - case e: SparkListenerSQLExecutionStart => filterExecutionStart(e) - case e: SparkListenerSQLAdaptiveExecutionUpdate => filterAdaptiveExecutionUpdate(e) - case e: SparkListenerSQLExecutionEnd => filterExecutionEnd(e) - case e: SparkListenerDriverAccumUpdates => filterDriverAccumUpdates(e) - - // these events are for finished batches so safer to ignore - case _: StreamingQueryListener.QueryProgressEvent => Some(false) - case _ => None - } - - def filterExecutionStart(event: SparkListenerSQLExecutionStart): Option[Boolean] = { - Some(liveExecutionToJobs.contains(event.executionId)) - } - - def filterAdaptiveExecutionUpdate( - event: SparkListenerSQLAdaptiveExecutionUpdate): Option[Boolean] = { - Some(liveExecutionToJobs.contains(event.executionId)) - } - - def filterExecutionEnd(event: SparkListenerSQLExecutionEnd): Option[Boolean] = { - Some(liveExecutionToJobs.contains(event.executionId)) - } + override def accept(event: SparkListenerEvent): Option[Boolean] = { + val fn: PartialFunction[SparkListenerEvent, Boolean] = { + case e: SparkListenerSQLExecutionStart => + liveExecutionToJobs.contains(e.executionId) + case e: SparkListenerSQLAdaptiveExecutionUpdate => + liveExecutionToJobs.contains(e.executionId) + case e: SparkListenerSQLExecutionEnd => + liveExecutionToJobs.contains(e.executionId) + case e: SparkListenerDriverAccumUpdates => + liveExecutionToJobs.contains(e.executionId) + + // these events are for finished batches so safer to ignore + case _: StreamingQueryListener.QueryProgressEvent => false + } + val optFn = fn.andThen { bool => Some(bool) } + val trueOrNoneJobEventFn = acceptFnForJobEvents.andThen { bool => trueOrNone(Some(bool)) } - def filterDriverAccumUpdates(event: SparkListenerDriverAccumUpdates): Option[Boolean] = { - Some(liveExecutionToJobs.contains(event.executionId)) + optFn.orElse(trueOrNoneJobEventFn).apply(event) } private def trueOrNone(booleanValue: Option[Boolean]): Option[Boolean] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala index 85331d032b294..6938bd7d91a37 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala @@ -39,12 +39,11 @@ class SQLLiveEntitiesEventFilterSuite extends SparkFunSuite { stageToTasks, stageToRDDs) // Verifying with finished SQL execution 1 - assert(filter.filterOtherEvent(SparkListenerSQLExecutionStart(1, "description1", "details1", + assert(filter.accept(SparkListenerSQLExecutionStart(1, "description1", "details1", "plan", null, 0)) === Some(false)) - assert(filter.filterOtherEvent(SparkListenerSQLExecutionEnd(1, 0)) === Some(false)) - assert(filter.filterOtherEvent(SparkListenerSQLAdaptiveExecutionUpdate(1, "plan", null)) - === Some(false)) - assert(filter.filterOtherEvent(SparkListenerDriverAccumUpdates(1, Seq.empty)) === Some(false)) + assert(filter.accept(SparkListenerSQLExecutionEnd(1, 0)) === Some(false)) + assert(filter.accept(SparkListenerSQLAdaptiveExecutionUpdate(1, "plan", null)) === Some(false)) + assert(filter.accept(SparkListenerDriverAccumUpdates(1, Seq.empty)) === Some(false)) // Verifying with finished job 1 val rddsForStage1 = createRddsWithId(1 to 2) @@ -59,40 +58,39 @@ class SQLLiveEntitiesEventFilterSuite extends SparkFunSuite { val unpersistRDDEventsForJob1 = (1 to 2).map(SparkListenerUnpersistRDD) // job events for finished job should be considered as "don't know" - assert(filter.filterJobStart(jobStartEventForJob1) === None) - assert(filter.filterJobEnd(jobEndEventForJob1) === None) + assert(filter.accept(jobStartEventForJob1) === None) + assert(filter.accept(jobEndEventForJob1) === None) // stage events for finished job should be considered as "don't know" - assert(filter.filterStageSubmitted(stageSubmittedEventsForJob1) === None) - assert(filter.filterStageCompleted(stageCompletedEventsForJob1) === None) + assert(filter.accept(stageSubmittedEventsForJob1) === None) + assert(filter.accept(stageCompletedEventsForJob1) === None) unpersistRDDEventsForJob1.foreach { event => - assert(filter.filterUnpersistRDD(event) === None) + assert(filter.accept(event) === None) } val taskSpeculativeTaskSubmittedEvent = SparkListenerSpeculativeTaskSubmitted(stage1.stageId, stageAttemptId = 1) - assert(filter.filterSpeculativeTaskSubmitted(taskSpeculativeTaskSubmittedEvent) === None) + assert(filter.accept(taskSpeculativeTaskSubmittedEvent) === None) // task events for finished job should be considered as "don't know" tasksForStage1.foreach { task => val taskStartEvent = SparkListenerTaskStart(stage1.stageId, 0, task) - assert(filter.filterTaskStart(taskStartEvent) === None) + assert(filter.accept(taskStartEvent) === None) val taskGettingResultEvent = SparkListenerTaskGettingResult(task) - assert(filter.filterTaskGettingResult(taskGettingResultEvent) === None) + assert(filter.accept(taskGettingResultEvent) === None) val taskEndEvent = SparkListenerTaskEnd(stage1.stageId, 0, "taskType", Success, task, new ExecutorMetrics, null) - assert(filter.filterTaskEnd(taskEndEvent) === None) + assert(filter.accept(taskEndEvent) === None) } // Verifying with live SQL execution 2 - assert(filter.filterOtherEvent(SparkListenerSQLExecutionStart(2, "description2", "details2", + assert(filter.accept(SparkListenerSQLExecutionStart(2, "description2", "details2", "plan", null, 0)) === Some(true)) - assert(filter.filterOtherEvent(SparkListenerSQLExecutionEnd(2, 0)) === Some(true)) - assert(filter.filterOtherEvent(SparkListenerSQLAdaptiveExecutionUpdate(2, "plan", null)) - === Some(true)) - assert(filter.filterOtherEvent(SparkListenerDriverAccumUpdates(2, Seq.empty)) === Some(true)) + assert(filter.accept(SparkListenerSQLExecutionEnd(2, 0)) === Some(true)) + assert(filter.accept(SparkListenerSQLAdaptiveExecutionUpdate(2, "plan", null)) === Some(true)) + assert(filter.accept(SparkListenerDriverAccumUpdates(2, Seq.empty)) === Some(true)) // Verifying with live job 2 val rddsForStage2 = createRddsWithId(3 to 4) @@ -106,30 +104,30 @@ class SQLLiveEntitiesEventFilterSuite extends SparkFunSuite { val unpersistRDDEventsForJob2 = rddsForStage2.map { rdd => SparkListenerUnpersistRDD(rdd.id) } // job events for live job should be filtered in - assert(filter.filterJobStart(jobStartEventForJob2) === Some(true)) + assert(filter.accept(jobStartEventForJob2) === Some(true)) // stage events for live job should be filtered in - assert(filter.filterStageSubmitted(stageSubmittedEventsForJob2) === Some(true)) - assert(filter.filterStageCompleted(stageCompletedEventsForJob2) === Some(true)) + assert(filter.accept(stageSubmittedEventsForJob2) === Some(true)) + assert(filter.accept(stageCompletedEventsForJob2) === Some(true)) unpersistRDDEventsForJob2.foreach { event => - assert(filter.filterUnpersistRDD(event) === Some(true)) + assert(filter.accept(event) === Some(true)) } val taskSpeculativeTaskSubmittedEvent2 = SparkListenerSpeculativeTaskSubmitted(stage2.stageId, stageAttemptId = 1) - assert(filter.filterSpeculativeTaskSubmitted(taskSpeculativeTaskSubmittedEvent2) === Some(true)) + assert(filter.accept(taskSpeculativeTaskSubmittedEvent2) === Some(true)) // task events for live job should be filtered in tasksForStage2.foreach { task => val taskStartEvent = SparkListenerTaskStart(stage2.stageId, 0, task) - assert(filter.filterTaskStart(taskStartEvent) === Some(true)) + assert(filter.accept(taskStartEvent) === Some(true)) val taskGettingResultEvent = SparkListenerTaskGettingResult(task) - assert(filter.filterTaskGettingResult(taskGettingResultEvent) === Some(true)) + assert(filter.accept(taskGettingResultEvent) === Some(true)) val taskEndEvent = SparkListenerTaskEnd(stage1.stageId, 0, "taskType", Success, task, new ExecutorMetrics, null) - assert(filter.filterTaskEnd(taskEndEvent) === Some(true)) + assert(filter.accept(taskEndEvent) === Some(true)) } } } From ef9f331a89f25cd3b2c198e900da59b265f3d8db Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 9 Dec 2019 23:44:34 +0900 Subject: [PATCH 21/28] Apply PartialFunction to EventFilter --- .../history/BasicEventFilterBuilder.scala | 20 ++--- .../spark/deploy/history/EventFilter.scala | 16 ++-- .../history/BasicEventFilterSuite.scala | 81 ++++++++++--------- .../deploy/history/EventLogTestHelper.scala | 24 +++--- .../history/SQLEventFilterBuilder.scala | 42 ++++------ .../SQLLiveEntitiesEventFilterSuite.scala | 51 ++++++------ 6 files changed, 114 insertions(+), 120 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala index 14d8a2bba9d50..cc52684e619ef 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala @@ -147,16 +147,16 @@ private[spark] class BasicEventFilter( logDebug(s"live executors : $liveExecutors") - override def accept(event: SparkListenerEvent): Option[Boolean] = { - val fn: PartialFunction[SparkListenerEvent, Boolean] = { - case e: SparkListenerExecutorAdded => liveExecutors.contains(e.executorId) - case e: SparkListenerExecutorRemoved => liveExecutors.contains(e.executorId) - case e: SparkListenerExecutorBlacklisted => liveExecutors.contains(e.executorId) - case e: SparkListenerExecutorUnblacklisted => liveExecutors.contains(e.executorId) - case e: SparkListenerStageExecutorMetrics => liveExecutors.contains(e.execId) - } - - fn.orElse(acceptFnForJobEvents).lift.apply(event) + private val _acceptFn: PartialFunction[SparkListenerEvent, Boolean] = { + case e: SparkListenerExecutorAdded => liveExecutors.contains(e.executorId) + case e: SparkListenerExecutorRemoved => liveExecutors.contains(e.executorId) + case e: SparkListenerExecutorBlacklisted => liveExecutors.contains(e.executorId) + case e: SparkListenerExecutorUnblacklisted => liveExecutors.contains(e.executorId) + case e: SparkListenerStageExecutorMetrics => liveExecutors.contains(e.execId) + } + + override def acceptFn(): PartialFunction[SparkListenerEvent, Boolean] = { + _acceptFn.orElse(acceptFnForJobEvents) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala index fbc238cb49137..3560c7d542a50 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala @@ -61,25 +61,21 @@ object EventFilterBuilder { } } -/** - * [[EventFilter]] decides whether the given event should be accepted or rejected. - * - * Please refer [[FilteredEventLogFileRewriter]] for more details on how the filter will be used. - */ +/** [[EventFilter]] decides whether the given event should be accepted or rejected. */ private[spark] trait EventFilter { /** * Classify whether the event is accepted or rejected by this filter. * - * Note that the method signature requires to return Option[Boolean] instead of Boolean: - * if the filter is able to decide with event whether it should be accepted or rejected, - * it can return either Some(true) or Some(false). Otherwise it should return None. + * The method should return the partial function which matches the events where the filter can + * decide whether the event should be accepted or rejected. Otherwise it should leave the events + * be unmatched. */ - def accept(event: SparkListenerEvent): Option[Boolean] + def acceptFn(): PartialFunction[SparkListenerEvent, Boolean] } object EventFilter extends Logging { def checkFilters(filters: Seq[EventFilter], event: SparkListenerEvent): Boolean = { - val results = filters.flatMap(_.accept(event)) + val results = filters.flatMap(_.acceptFn().lift.apply(event)) results.isEmpty || results.forall(_ == true) } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala index f71f4f1840f7b..1578ff2219101 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala @@ -35,6 +35,7 @@ class BasicEventFilterSuite extends SparkFunSuite { val liveExecutors: Set[String] = Set("1", "2") val filter = new BasicEventFilter(liveJobToStages, stageToTasks, stageToRDDs, liveExecutors) + val acceptFn = filter.acceptFn().lift // Verifying with finished job 1 val rddsForStage1 = createRddsWithId(1 to 2) @@ -49,12 +50,12 @@ class BasicEventFilterSuite extends SparkFunSuite { val unpersistRDDEventsForJob1 = (1 to 2).map(SparkListenerUnpersistRDD) // job events for finished job should be filtered out - assertFilterJobEvents(filter, jobStartEventForJob1, jobEndEventForJob1, Some(false)) + assertFilterJobEvents(acceptFn, jobStartEventForJob1, jobEndEventForJob1, Some(false)) // stage events for finished job should be filtered out // NOTE: it doesn't filter out stage events which are also related to the executor assertFilterStageEvents( - filter, + acceptFn, stageSubmittedEventsForJob1, stageCompletedEventsForJob1, unpersistRDDEventsForJob1, @@ -62,7 +63,7 @@ class BasicEventFilterSuite extends SparkFunSuite { Some(false)) // task events for finished job should be filtered out - assertFilterTaskEvents(filter, tasksForStage1, stage1, Some(false)) + assertFilterTaskEvents(acceptFn, tasksForStage1, stage1, Some(false)) // Verifying with live job 2 val rddsForStage2 = createRddsWithId(3 to 4) @@ -76,11 +77,11 @@ class BasicEventFilterSuite extends SparkFunSuite { val unpersistRDDEventsForJob2 = rddsForStage2.map { rdd => SparkListenerUnpersistRDD(rdd.id) } // job events for live job should be filtered in - assert(filter.accept(jobStartEventForJob2) === Some(true)) + assert(acceptFn(jobStartEventForJob2) === Some(true)) // stage events for live job should be filtered in assertFilterStageEvents( - filter, + acceptFn, stageSubmittedEventsForJob2, stageCompletedEventsForJob2, unpersistRDDEventsForJob2, @@ -88,7 +89,7 @@ class BasicEventFilterSuite extends SparkFunSuite { Some(true)) // task events for live job should be filtered in - assertFilterTaskEvents(filter, tasksForStage2, stage2, Some(true)) + assertFilterTaskEvents(acceptFn, tasksForStage2, stage2, Some(true)) } test("filter out events for dead executors") { @@ -96,26 +97,31 @@ class BasicEventFilterSuite extends SparkFunSuite { val liveExecutors: Set[String] = Set("2") val filter = new BasicEventFilter(Map.empty, Map.empty, Map.empty, liveExecutors) + val acceptFn = filter.acceptFn().lift // events for dead executor should be filtered out - assert(filter.accept(createExecutorAddedEvent(1)) === Some(false)) + assert(acceptFn(createExecutorAddedEvent(1)) === Some(false)) // though the name of event is stage executor metrics, AppStatusListener only deals with // live executors - assert(filter.accept( + assert(acceptFn( SparkListenerStageExecutorMetrics(1.toString, 0, 0, new ExecutorMetrics)) === Some(false)) - assert(filter.accept(SparkListenerExecutorBlacklisted(0, 1.toString, 1)) === Some(false)) - assert(filter.accept(SparkListenerExecutorUnblacklisted(0, 1.toString)) === Some(false)) - assert(filter.accept(createExecutorRemovedEvent(1)) === Some(false)) + assert(acceptFn(SparkListenerExecutorBlacklisted(0, 1.toString, 1)) === + Some(false)) + assert(acceptFn(SparkListenerExecutorUnblacklisted(0, 1.toString)) === + Some(false)) + assert(acceptFn(createExecutorRemovedEvent(1)) === Some(false)) // events for live executor should be filtered in - assert(filter.accept(createExecutorAddedEvent(2)) === Some(true)) - assert(filter.accept( + assert(acceptFn(createExecutorAddedEvent(2)) === Some(true)) + assert(acceptFn( SparkListenerStageExecutorMetrics(2.toString, 0, 0, new ExecutorMetrics)) === Some(true)) - assert(filter.accept(SparkListenerExecutorBlacklisted(0, 2.toString, 1)) === Some(true)) - assert(filter.accept(SparkListenerExecutorUnblacklisted(0, 2.toString)) === Some(true)) - assert(filter.accept(createExecutorRemovedEvent(2)) === Some(true)) + assert(acceptFn(SparkListenerExecutorBlacklisted(0, 2.toString, 1)) === + Some(true)) + assert(acceptFn(SparkListenerExecutorUnblacklisted(0, 2.toString)) === + Some(true)) + assert(acceptFn(createExecutorRemovedEvent(2)) === Some(true)) } test("other events should be left to other filters") { @@ -124,59 +130,60 @@ class BasicEventFilterSuite extends SparkFunSuite { } val filter = new BasicEventFilter(Map.empty, Map.empty, Map.empty, Set.empty) + val acceptFn = filter.acceptFn().lift - assertNone(filter.accept(SparkListenerEnvironmentUpdate(Map.empty))) - assertNone(filter.accept(SparkListenerApplicationStart("1", Some("1"), 0, "user", None))) - assertNone(filter.accept(SparkListenerApplicationEnd(1))) + assertNone(acceptFn(SparkListenerEnvironmentUpdate(Map.empty))) + assertNone(acceptFn(SparkListenerApplicationStart("1", Some("1"), 0, "user", None))) + assertNone(acceptFn(SparkListenerApplicationEnd(1))) val bmId = BlockManagerId("1", "host1", 1) - assertNone(filter.accept(SparkListenerBlockManagerAdded(0, bmId, 1))) - assertNone(filter.accept(SparkListenerBlockManagerRemoved(1, bmId))) - assertNone(filter.accept(SparkListenerBlockUpdated( + assertNone(acceptFn(SparkListenerBlockManagerAdded(0, bmId, 1))) + assertNone(acceptFn(SparkListenerBlockManagerRemoved(1, bmId))) + assertNone(acceptFn(SparkListenerBlockUpdated( storage.BlockUpdatedInfo(bmId, RDDBlockId(1, 1), StorageLevel.DISK_ONLY, 0, 10)))) - assertNone(filter.accept(SparkListenerNodeBlacklisted(0, "host1", 1))) - assertNone(filter.accept(SparkListenerNodeUnblacklisted(0, "host1"))) - assertNone(filter.accept(SparkListenerLogStart("testVersion"))) + assertNone(acceptFn(SparkListenerNodeBlacklisted(0, "host1", 1))) + assertNone(acceptFn(SparkListenerNodeUnblacklisted(0, "host1"))) + assertNone(acceptFn(SparkListenerLogStart("testVersion"))) } private def assertFilterJobEvents( - filter: BasicEventFilter, + acceptFn: SparkListenerEvent => Option[Boolean], jobStart: SparkListenerJobStart, jobEnd: SparkListenerJobEnd, expectedVal: Option[Boolean]): Unit = { - assert(filter.accept(jobStart) === expectedVal) - assert(filter.accept(jobEnd) === expectedVal) + assert(acceptFn(jobStart) === expectedVal) + assert(acceptFn(jobEnd) === expectedVal) } private def assertFilterStageEvents( - filter: BasicEventFilter, + acceptFn: SparkListenerEvent => Option[Boolean], stageSubmitted: SparkListenerStageSubmitted, stageCompleted: SparkListenerStageCompleted, unpersistRDDs: Seq[SparkListenerUnpersistRDD], taskSpeculativeSubmitted: SparkListenerSpeculativeTaskSubmitted, expectedVal: Option[Boolean]): Unit = { - assert(filter.accept(stageSubmitted) === expectedVal) - assert(filter.accept(stageCompleted) === expectedVal) + assert(acceptFn(stageSubmitted) === expectedVal) + assert(acceptFn(stageCompleted) === expectedVal) unpersistRDDs.foreach { event => - assert(filter.accept(event) === expectedVal) + assert(acceptFn(event) === expectedVal) } - assert(filter.accept(taskSpeculativeSubmitted) === expectedVal) + assert(acceptFn(taskSpeculativeSubmitted) === expectedVal) } private def assertFilterTaskEvents( - filter: BasicEventFilter, + acceptFn: SparkListenerEvent => Option[Boolean], taskInfos: Seq[TaskInfo], stageInfo: StageInfo, expectedVal: Option[Boolean]): Unit = { taskInfos.foreach { task => val taskStartEvent = SparkListenerTaskStart(stageInfo.stageId, 0, task) - assert(filter.accept(taskStartEvent) === expectedVal) + assert(acceptFn(taskStartEvent) === expectedVal) val taskGettingResultEvent = SparkListenerTaskGettingResult(task) - assert(filter.accept(taskGettingResultEvent) === expectedVal) + assert(acceptFn(taskGettingResultEvent) === expectedVal) val taskEndEvent = SparkListenerTaskEnd(stageInfo.stageId, 0, "taskType", Success, task, new ExecutorMetrics, null) - assert(filter.accept(taskEndEvent) === expectedVal) + assert(acceptFn(taskEndEvent) === expectedVal) } } } 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 a119abf9b4335..0ac8fad53bdd9 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 @@ -82,23 +82,21 @@ object EventLogTestHelper { } class TestEventFilter1 extends EventFilter { - override def accept(event: SparkListenerEvent): Option[Boolean] = event match { - case _: SparkListenerApplicationEnd => Some(true) - case _: SparkListenerBlockManagerAdded => Some(true) - case _: SparkListenerApplicationStart => Some(false) - case _ => None + override def acceptFn(): PartialFunction[SparkListenerEvent, Boolean] = { + case _: SparkListenerApplicationEnd => true + case _: SparkListenerBlockManagerAdded => true + case _: SparkListenerApplicationStart => false } } class TestEventFilter2 extends EventFilter { - override def accept(event: SparkListenerEvent): Option[Boolean] = event match { - case _: SparkListenerApplicationEnd => Some(true) - case _: SparkListenerEnvironmentUpdate => Some(true) - case _: SparkListenerNodeBlacklisted => Some(true) - case _: SparkListenerBlockManagerAdded => Some(false) - case _: SparkListenerApplicationStart => Some(false) - case _: SparkListenerNodeUnblacklisted => Some(false) - case _ => None + override def acceptFn(): PartialFunction[SparkListenerEvent, Boolean] = { + case _: SparkListenerApplicationEnd => true + case _: SparkListenerEnvironmentUpdate => true + case _: SparkListenerNodeBlacklisted => true + case _: SparkListenerBlockManagerAdded => false + case _: SparkListenerApplicationStart => false + case _: SparkListenerNodeUnblacklisted => false } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala index 8cedd4d7be31d..fad9b7d1eba21 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala @@ -126,33 +126,25 @@ private[spark] class SQLLiveEntitiesEventFilter( logDebug(s"live executions : ${liveExecutionToJobs.keySet}") logDebug(s"jobs in live executions : ${liveExecutionToJobs.values.flatten}") - override def accept(event: SparkListenerEvent): Option[Boolean] = { - val fn: PartialFunction[SparkListenerEvent, Boolean] = { - case e: SparkListenerSQLExecutionStart => - liveExecutionToJobs.contains(e.executionId) - case e: SparkListenerSQLAdaptiveExecutionUpdate => - liveExecutionToJobs.contains(e.executionId) - case e: SparkListenerSQLExecutionEnd => - liveExecutionToJobs.contains(e.executionId) - case e: SparkListenerDriverAccumUpdates => - liveExecutionToJobs.contains(e.executionId) - - // these events are for finished batches so safer to ignore - case _: StreamingQueryListener.QueryProgressEvent => false - } - val optFn = fn.andThen { bool => Some(bool) } - val trueOrNoneJobEventFn = acceptFnForJobEvents.andThen { bool => trueOrNone(Some(bool)) } - - optFn.orElse(trueOrNoneJobEventFn).apply(event) + private val _acceptFn: PartialFunction[SparkListenerEvent, Boolean] = { + case e: SparkListenerSQLExecutionStart => + liveExecutionToJobs.contains(e.executionId) + case e: SparkListenerSQLAdaptiveExecutionUpdate => + liveExecutionToJobs.contains(e.executionId) + case e: SparkListenerSQLExecutionEnd => + liveExecutionToJobs.contains(e.executionId) + case e: SparkListenerDriverAccumUpdates => + liveExecutionToJobs.contains(e.executionId) + + case e if acceptFnForJobEvents.isDefinedAt(e) && acceptFnForJobEvents(e) => + // if acceptFnForJobEvents(e) returns false, we should leave it to "unmatched" + true + + // these events are for finished batches so safer to ignore + case _: StreamingQueryListener.QueryProgressEvent => false } - private def trueOrNone(booleanValue: Option[Boolean]): Option[Boolean] = { - booleanValue match { - case Some(true) => Some(true) - case Some(false) => None - case None => None - } - } + override def acceptFn(): PartialFunction[SparkListenerEvent, Boolean] = _acceptFn } private[spark] object SQLLiveEntitiesEventFilter { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala index 6938bd7d91a37..a3ba4ce9056f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala @@ -37,13 +37,14 @@ class SQLLiveEntitiesEventFilterSuite extends SparkFunSuite { val filter = new SQLLiveEntitiesEventFilter(liveExecutionToJobs, liveJobToStages, stageToTasks, stageToRDDs) + val acceptFn = filter.acceptFn().lift // Verifying with finished SQL execution 1 - assert(filter.accept(SparkListenerSQLExecutionStart(1, "description1", "details1", + assert(acceptFn(SparkListenerSQLExecutionStart(1, "description1", "details1", "plan", null, 0)) === Some(false)) - assert(filter.accept(SparkListenerSQLExecutionEnd(1, 0)) === Some(false)) - assert(filter.accept(SparkListenerSQLAdaptiveExecutionUpdate(1, "plan", null)) === Some(false)) - assert(filter.accept(SparkListenerDriverAccumUpdates(1, Seq.empty)) === Some(false)) + assert(acceptFn(SparkListenerSQLExecutionEnd(1, 0)) === Some(false)) + assert(acceptFn(SparkListenerSQLAdaptiveExecutionUpdate(1, "plan", null)) === Some(false)) + assert(acceptFn(SparkListenerDriverAccumUpdates(1, Seq.empty)) === Some(false)) // Verifying with finished job 1 val rddsForStage1 = createRddsWithId(1 to 2) @@ -58,39 +59,39 @@ class SQLLiveEntitiesEventFilterSuite extends SparkFunSuite { val unpersistRDDEventsForJob1 = (1 to 2).map(SparkListenerUnpersistRDD) // job events for finished job should be considered as "don't know" - assert(filter.accept(jobStartEventForJob1) === None) - assert(filter.accept(jobEndEventForJob1) === None) + assert(acceptFn(jobStartEventForJob1) === None) + assert(acceptFn(jobEndEventForJob1) === None) // stage events for finished job should be considered as "don't know" - assert(filter.accept(stageSubmittedEventsForJob1) === None) - assert(filter.accept(stageCompletedEventsForJob1) === None) + assert(acceptFn(stageSubmittedEventsForJob1) === None) + assert(acceptFn(stageCompletedEventsForJob1) === None) unpersistRDDEventsForJob1.foreach { event => - assert(filter.accept(event) === None) + assert(acceptFn(event) === None) } val taskSpeculativeTaskSubmittedEvent = SparkListenerSpeculativeTaskSubmitted(stage1.stageId, stageAttemptId = 1) - assert(filter.accept(taskSpeculativeTaskSubmittedEvent) === None) + assert(acceptFn(taskSpeculativeTaskSubmittedEvent) === None) // task events for finished job should be considered as "don't know" tasksForStage1.foreach { task => val taskStartEvent = SparkListenerTaskStart(stage1.stageId, 0, task) - assert(filter.accept(taskStartEvent) === None) + assert(acceptFn(taskStartEvent) === None) val taskGettingResultEvent = SparkListenerTaskGettingResult(task) - assert(filter.accept(taskGettingResultEvent) === None) + assert(acceptFn(taskGettingResultEvent) === None) val taskEndEvent = SparkListenerTaskEnd(stage1.stageId, 0, "taskType", Success, task, new ExecutorMetrics, null) - assert(filter.accept(taskEndEvent) === None) + assert(acceptFn(taskEndEvent) === None) } // Verifying with live SQL execution 2 - assert(filter.accept(SparkListenerSQLExecutionStart(2, "description2", "details2", + assert(acceptFn(SparkListenerSQLExecutionStart(2, "description2", "details2", "plan", null, 0)) === Some(true)) - assert(filter.accept(SparkListenerSQLExecutionEnd(2, 0)) === Some(true)) - assert(filter.accept(SparkListenerSQLAdaptiveExecutionUpdate(2, "plan", null)) === Some(true)) - assert(filter.accept(SparkListenerDriverAccumUpdates(2, Seq.empty)) === Some(true)) + assert(acceptFn(SparkListenerSQLExecutionEnd(2, 0)) === Some(true)) + assert(acceptFn(SparkListenerSQLAdaptiveExecutionUpdate(2, "plan", null)) === Some(true)) + assert(acceptFn(SparkListenerDriverAccumUpdates(2, Seq.empty)) === Some(true)) // Verifying with live job 2 val rddsForStage2 = createRddsWithId(3 to 4) @@ -104,30 +105,30 @@ class SQLLiveEntitiesEventFilterSuite extends SparkFunSuite { val unpersistRDDEventsForJob2 = rddsForStage2.map { rdd => SparkListenerUnpersistRDD(rdd.id) } // job events for live job should be filtered in - assert(filter.accept(jobStartEventForJob2) === Some(true)) + assert(acceptFn(jobStartEventForJob2) === Some(true)) // stage events for live job should be filtered in - assert(filter.accept(stageSubmittedEventsForJob2) === Some(true)) - assert(filter.accept(stageCompletedEventsForJob2) === Some(true)) + assert(acceptFn(stageSubmittedEventsForJob2) === Some(true)) + assert(acceptFn(stageCompletedEventsForJob2) === Some(true)) unpersistRDDEventsForJob2.foreach { event => - assert(filter.accept(event) === Some(true)) + assert(acceptFn(event) === Some(true)) } val taskSpeculativeTaskSubmittedEvent2 = SparkListenerSpeculativeTaskSubmitted(stage2.stageId, stageAttemptId = 1) - assert(filter.accept(taskSpeculativeTaskSubmittedEvent2) === Some(true)) + assert(acceptFn(taskSpeculativeTaskSubmittedEvent2) === Some(true)) // task events for live job should be filtered in tasksForStage2.foreach { task => val taskStartEvent = SparkListenerTaskStart(stage2.stageId, 0, task) - assert(filter.accept(taskStartEvent) === Some(true)) + assert(acceptFn(taskStartEvent) === Some(true)) val taskGettingResultEvent = SparkListenerTaskGettingResult(task) - assert(filter.accept(taskGettingResultEvent) === Some(true)) + assert(acceptFn(taskGettingResultEvent) === Some(true)) val taskEndEvent = SparkListenerTaskEnd(stage1.stageId, 0, "taskType", Success, task, new ExecutorMetrics, null) - assert(filter.accept(taskEndEvent) === Some(true)) + assert(acceptFn(taskEndEvent) === Some(true)) } } } From ed2b1e273b96389697b3583d3cf515e32b9bb806 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 10 Dec 2019 07:01:01 +0900 Subject: [PATCH 22/28] Refine comments --- .../deploy/history/BasicEventFilterBuilder.scala | 14 +++++++------- .../spark/deploy/history/EventFilter.scala | 3 +-- .../deploy/history/EventLogFileCompactor.scala | 10 +++++----- .../deploy/history/BasicEventFilterSuite.scala | 16 ++++++++-------- .../history/EventLogFileCompactorSuite.scala | 4 ++-- .../EventLogFilterRateCalculatorSuite.scala | 16 ++++++++-------- .../deploy/history/FsHistoryProviderSuite.scala | 8 ++++---- .../history/SQLEventFilterBuilder.scala | 12 ++++++------ .../SQLLiveEntitiesEventFilterSuite.scala | 6 +++--- 9 files changed, 44 insertions(+), 45 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala index cc52684e619ef..f4ddadd1dc298 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala @@ -24,8 +24,8 @@ import org.apache.spark.scheduler._ /** * This class tracks both live jobs and live executors, and pass the list to the - * [[BasicEventFilter]] to help BasicEventFilter to filter out finished jobs - * (+ stages/tasks/RDDs) and dead executors. + * [[BasicEventFilter]] to help BasicEventFilter to reject finished jobs (+ stages/tasks/RDDs) + * and dead executors. */ private[spark] class BasicEventFilterBuilder extends SparkListener with EventFilterBuilder { private val _liveJobToStages = new mutable.HashMap[Int, Seq[Int]] @@ -72,10 +72,10 @@ private[spark] class BasicEventFilterBuilder extends SparkListener with EventFil } /** - * This class provides the functionality to filter out events which are related to the finished - * jobs based on the given information. This class only deals with job related events, and returns - * either Some(true) or Some(false) - successors should override the methods if they don't want to - * return Some(false) for finished jobs and related events. + * This class provides the functionality to reject events which are related to the finished + * jobs based on the given information. This class only deals with job related events, and provides + * a PartialFunction which returns false for rejected events for finished jobs, returns true + * otherwise. */ private[spark] abstract class JobEventFilter( jobToStages: Map[Int, Seq[Int]], @@ -134,7 +134,7 @@ private[spark] abstract class JobEventFilter( } /** - * This class filters out events which are related to the finished jobs or dead executors, + * This class rejects events which are related to the finished jobs or dead executors, * based on the given information. The events which are not related to the job and executor * will be considered as "Don't mind". */ diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala index 3560c7d542a50..e503cd0284c44 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala @@ -33,7 +33,7 @@ import org.apache.spark.util.{JsonProtocol, Utils} /** * EventFilterBuilder provides the interface to gather the information from events being received * by [[SparkListenerInterface]], and create a new [[EventFilter]] instance which leverages - * information gathered to decide whether the event should be filtered or not. + * information gathered to decide whether the event should be accepted or not. */ private[spark] trait EventFilterBuilder extends SparkListenerInterface { def createFilter(): EventFilter @@ -104,7 +104,6 @@ object EventFilter extends Logging { Some(JsonProtocol.sparkEventFromJson(parse(currentLine))) } catch { // ignore any exception occurred from unidentified json - // just skip handling and write the line case NonFatal(_) => fnUnidentified(currentLine) None diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index 792f122c57c01..b2d29e7c0ed6a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -34,8 +34,8 @@ import org.apache.spark.scheduler._ * 1) Initialize available [[EventFilterBuilder]] instances, and replay the old event log files with * builders, so that these builders can gather the information to create [[EventFilter]] instances. * 2) Initialize [[EventFilter]] instances from [[EventFilterBuilder]] instances, and replay the - * old event log files with filters. Rewrite the content to the compact file if the filters decide - * to filter in. + * old event log files with filters. Rewrite the events to the compact file which the filters decide + * to accept. * * This class assumes caller will provide the sorted list of files which are sorted by the index of * event log file - caller should keep in mind that this class doesn't care about the semantic of @@ -106,8 +106,8 @@ class EventLogFileCompactor( /** * This class rewrites the event log files into one compact file: the compact file will only - * contain the events which pass the filters. Events will be filtered out only when all filters - * decide to filter out the event or don't mind about the event. Otherwise, the original line for + * contain the events which pass the filters. Events will be dropped only when all filters + * decide to reject the event or don't mind about the event. Otherwise, the original line for * the event is written to the compact file as it is. */ class FilteredEventLogFileRewriter( @@ -140,7 +140,7 @@ class FilteredEventLogFileRewriter( } /** - * This class helps to write compact file; to avoid reimplement everything, it extends + * This class helps to write compact file; to avoid reimplementing everything, it extends * [[SingleEventLogFileWriter]], but only `originalFilePath` is used to determine the * path of compact file. */ diff --git a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala index 1578ff2219101..0f742fb57ab96 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala @@ -49,10 +49,10 @@ class BasicEventFilterSuite extends SparkFunSuite { val stageCompletedEventsForJob1 = SparkListenerStageCompleted(stage1) val unpersistRDDEventsForJob1 = (1 to 2).map(SparkListenerUnpersistRDD) - // job events for finished job should be filtered out + // job events for finished job should be rejected assertFilterJobEvents(acceptFn, jobStartEventForJob1, jobEndEventForJob1, Some(false)) - // stage events for finished job should be filtered out + // stage events for finished job should be rejected // NOTE: it doesn't filter out stage events which are also related to the executor assertFilterStageEvents( acceptFn, @@ -62,7 +62,7 @@ class BasicEventFilterSuite extends SparkFunSuite { SparkListenerSpeculativeTaskSubmitted(stage1.stageId, stageAttemptId = 1), Some(false)) - // task events for finished job should be filtered out + // task events for finished job should be rejected assertFilterTaskEvents(acceptFn, tasksForStage1, stage1, Some(false)) // Verifying with live job 2 @@ -76,10 +76,10 @@ class BasicEventFilterSuite extends SparkFunSuite { val stageCompletedEventsForJob2 = SparkListenerStageCompleted(stage2) val unpersistRDDEventsForJob2 = rddsForStage2.map { rdd => SparkListenerUnpersistRDD(rdd.id) } - // job events for live job should be filtered in + // job events for live job should be accepted assert(acceptFn(jobStartEventForJob2) === Some(true)) - // stage events for live job should be filtered in + // stage events for live job should be accepted assertFilterStageEvents( acceptFn, stageSubmittedEventsForJob2, @@ -88,7 +88,7 @@ class BasicEventFilterSuite extends SparkFunSuite { SparkListenerSpeculativeTaskSubmitted(stage2.stageId, stageAttemptId = 1), Some(true)) - // task events for live job should be filtered in + // task events for live job should be accepted assertFilterTaskEvents(acceptFn, tasksForStage2, stage2, Some(true)) } @@ -99,7 +99,7 @@ class BasicEventFilterSuite extends SparkFunSuite { val filter = new BasicEventFilter(Map.empty, Map.empty, Map.empty, liveExecutors) val acceptFn = filter.acceptFn().lift - // events for dead executor should be filtered out + // events for dead executor should be rejected assert(acceptFn(createExecutorAddedEvent(1)) === Some(false)) // though the name of event is stage executor metrics, AppStatusListener only deals with // live executors @@ -112,7 +112,7 @@ class BasicEventFilterSuite extends SparkFunSuite { Some(false)) assert(acceptFn(createExecutorRemovedEvent(1)) === Some(false)) - // events for live executor should be filtered in + // events for live executor should be accepted assert(acceptFn(createExecutorAddedEvent(2)) === Some(true)) assert(acceptFn( SparkListenerStageExecutorMetrics(2.toString, 0, 0, new ExecutorMetrics)) === diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala index 248eb7bcfa970..9f53976ea1319 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala @@ -117,7 +117,7 @@ class EventLogFileCompactorSuite extends SparkFunSuite { } } - test("events for finished job are filtered out in new compact file") { + test("events for finished job are dropped in new compact file") { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) @@ -142,7 +142,7 @@ class EventLogFileCompactorSuite extends SparkFunSuite { Utils.tryWithResource(EventLogFileReader.openEventLog(compactFilePath, fs)) { is => val lines = Source.fromInputStream(is)(Codec.UTF8).getLines().toList - assert(lines.length === 2, "Compacted file should have only two events being filtered in") + assert(lines.length === 2, "Compacted file should have only two events being accepted") lines.foreach { line => val event = JsonProtocol.sparkEventFromJson(parse(line)) assert(!event.isInstanceOf[SparkListenerJobStart] && diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculatorSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculatorSuite.scala index 304783450bf1e..a75f3ae0ba488 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculatorSuite.scala @@ -31,28 +31,28 @@ class EventLogFilterRateCalculatorSuite extends SparkFunSuite { private val sparkConf = new SparkConf() private val hadoopConf = SparkHadoopUtil.newConfiguration(sparkConf) - test("calculate filter-in rate") { + test("calculate accept rate") { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) val events = new mutable.ArrayBuffer[SparkListenerEvent] - // filterApplicationEnd: Some(true) & Some(true) => filter in + // filterApplicationEnd: Some(true) & Some(true) => accepted events += SparkListenerApplicationEnd(0) - // filterBlockManagerAdded: Some(true) & Some(false) => filter out + // filterBlockManagerAdded: Some(true) & Some(false) => rejected events += SparkListenerBlockManagerAdded(0, BlockManagerId("1", "host1", 1), 10) - // filterApplicationStart: Some(false) & Some(false) => filter out + // filterApplicationStart: Some(false) & Some(false) => rejected events += SparkListenerApplicationStart("app", None, 0, "user", None) - // filterNodeBlacklisted: None & Some(true) => filter in + // filterNodeBlacklisted: None & Some(true) => accepted events += SparkListenerNodeBlacklisted(0, "host1", 1) - // filterNodeUnblacklisted: None & Some(false) => filter out + // filterNodeUnblacklisted: None & Some(false) => rejected events += SparkListenerNodeUnblacklisted(0, "host1") - // other events: None & None => filter in + // other events: None & None => accepted events += SparkListenerUnpersistRDD(0) val logPath = EventLogTestHelper.writeEventLogFile(sparkConf, hadoopConf, dir, 1, events) @@ -61,7 +61,7 @@ class EventLogFilterRateCalculatorSuite extends SparkFunSuite { val filters = Seq(new TestEventFilter1, new TestEventFilter2) val calculator = new EventLogFilterRateCalculator(fs) - // 6 filtered in, 6 filtered out + // 6 accepted, 6 rejected assert(0.5d === calculator.doCalculate(logs, filters)) } } 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 f2ea999356630..b46140b92a67e 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 @@ -1449,11 +1449,11 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { provider.getOrUpdateCompactible(reader1) assert(1 === provider.checkEligibilityForCompactionCall) - // All events in log file are expected to be filtered in - so 'compactible' should be true, + // All events in log file are expected to be accepted - so 'compactible' should be true, // but we don't decide if there's only one file given the file might be updated later. assertCompactibleInLogInfo(provider, reader1, None) - // 10 of 11 events will be filtered out which meets the condition of compaction + // 10 of 11 events will be rejected which meets the condition of compaction val logPath2 = EventLogTestHelper.writeEventLogFile(conf, hadoopConf, dir, 2, (1 to 5).flatMap { idx => Seq(SparkListenerJobStart(idx, 1L, Seq.empty, null), @@ -1468,7 +1468,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { provider.getOrUpdateCompactible(reader2) assert(1 === provider.checkEligibilityForCompactionCall) - // All events in log file are expected to be filtered in - so 'compactible' should be false. + // All events in log file are expected to be accepted - so 'compactible' should be false. assertCompactibleInLogInfo(provider, reader2, Some(false)) val logPath3 = EventLogTestHelper.writeEventLogFile(conf, hadoopConf, dir, 3, Seq( @@ -1482,7 +1482,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { provider.getOrUpdateCompactible(reader3) assert(1 === provider.checkEligibilityForCompactionCall) - // Most of events in log file are expected to be filtered out - so 'compactible' should + // Most of events in log file are expected to be rejected - so 'compactible' should // be true. assertCompactibleInLogInfo(provider, reader3, Some(true)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala index fad9b7d1eba21..4fdfb3cea0403 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.streaming.StreamingQueryListener /** * This class tracks live SQL executions, and pass the list to the [[SQLLiveEntitiesEventFilter]] - * to help SQLLiveEntitiesEventFilter to filter out finished SQL executions as well as relevant + * to help SQLLiveEntitiesEventFilter to reject finished SQL executions as well as relevant * jobs (+ stages/tasks/RDDs). Unlike BasicEventFilterBuilder, it doesn't concern about the status * of individual job - it only concerns whether SQL execution is finished or not. */ @@ -105,12 +105,12 @@ private[spark] class SQLEventFilterBuilder extends SparkListener with EventFilte } /** - * This class filters out events which are related to the finished SQL executions based on the + * This class rejects events which are related to the finished SQL executions based on the * given information. * - * Note that filterXXX methods will return None instead of Some(false) if the event is related to - * job but not coupled with live SQL executions, because the instance has the information about - * jobs for live SQL executions which should be filtered in, but don't know whether the job is + * Note that acceptFn will not match the event instead of returning false if the event is related + * to job but not coupled with live SQL executions, because the instance has the information about + * jobs for live SQL executions which should be accepted, but don't know whether the job is * related to the finished SQL executions, or job is NOT related to the SQL executions. For this * case, it just gives up the decision and let other filters decide it. * @@ -137,7 +137,7 @@ private[spark] class SQLLiveEntitiesEventFilter( liveExecutionToJobs.contains(e.executionId) case e if acceptFnForJobEvents.isDefinedAt(e) && acceptFnForJobEvents(e) => - // if acceptFnForJobEvents(e) returns false, we should leave it to "unmatched" + // NOTE: if acceptFnForJobEvents(e) returns false, we should leave it to "unmatched" true // these events are for finished batches so safer to ignore diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala index a3ba4ce9056f0..a81afd0eefcc2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala @@ -104,10 +104,10 @@ class SQLLiveEntitiesEventFilterSuite extends SparkFunSuite { val stageCompletedEventsForJob2 = SparkListenerStageCompleted(stage2) val unpersistRDDEventsForJob2 = rddsForStage2.map { rdd => SparkListenerUnpersistRDD(rdd.id) } - // job events for live job should be filtered in + // job events for live job should be accepted assert(acceptFn(jobStartEventForJob2) === Some(true)) - // stage events for live job should be filtered in + // stage events for live job should be accepted assert(acceptFn(stageSubmittedEventsForJob2) === Some(true)) assert(acceptFn(stageCompletedEventsForJob2) === Some(true)) unpersistRDDEventsForJob2.foreach { event => @@ -118,7 +118,7 @@ class SQLLiveEntitiesEventFilterSuite extends SparkFunSuite { stageAttemptId = 1) assert(acceptFn(taskSpeculativeTaskSubmittedEvent2) === Some(true)) - // task events for live job should be filtered in + // task events for live job should be accepted tasksForStage2.foreach { task => val taskStartEvent = SparkListenerTaskStart(stage2.stageId, 0, task) assert(acceptFn(taskStartEvent) === Some(true)) From 7db7e78237365bd421a830fe6a57f7f8640dd14b Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 10 Dec 2019 12:58:00 +0900 Subject: [PATCH 23/28] Address review comments --- .../history/BasicEventFilterBuilder.scala | 16 +++++----------- .../spark/deploy/history/EventFilter.scala | 6 +++--- .../deploy/history/EventLogFileCompactor.scala | 6 +++--- docs/configuration.md | 6 +++--- .../history/SQLEventFilterBuilder.scala | 17 ++++++++--------- 5 files changed, 22 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala index f4ddadd1dc298..74ff0e2e45ecf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala @@ -68,7 +68,7 @@ private[spark] class BasicEventFilterBuilder extends SparkListener with EventFil _liveExecutors -= executorRemoved.executorId } - override def createFilter(): EventFilter = BasicEventFilter(this) + override def createFilter(): EventFilter = new BasicEventFilter(this) } /** @@ -145,6 +145,10 @@ private[spark] class BasicEventFilter( liveExecutors: Set[String]) extends JobEventFilter(_liveJobToStages, _stageToTasks, _stageToRDDs) with Logging { + def this(builder: BasicEventFilterBuilder) = { + this(builder.liveJobToStages, builder.stageToTasks, builder.stageToRDDs, builder.liveExecutors) + } + logDebug(s"live executors : $liveExecutors") private val _acceptFn: PartialFunction[SparkListenerEvent, Boolean] = { @@ -159,13 +163,3 @@ private[spark] class BasicEventFilter( _acceptFn.orElse(acceptFnForJobEvents) } } - -private[spark] object BasicEventFilter { - def apply(builder: BasicEventFilterBuilder): BasicEventFilter = { - new BasicEventFilter( - builder.liveJobToStages, - builder.stageToTasks, - builder.stageToRDDs, - builder.liveExecutors) - } -} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala index e503cd0284c44..084ce7d849c6d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala @@ -74,9 +74,9 @@ private[spark] trait EventFilter { } object EventFilter extends Logging { - def checkFilters(filters: Seq[EventFilter], event: SparkListenerEvent): Boolean = { + def acceptEvent(filters: Seq[EventFilter], event: SparkListenerEvent): Boolean = { val results = filters.flatMap(_.acceptFn().lift.apply(event)) - results.isEmpty || results.forall(_ == true) + results.isEmpty || !results.contains(false) } def applyFilterToFile( @@ -110,7 +110,7 @@ object EventFilter extends Logging { } event.foreach { e => - if (EventFilter.checkFilters(filters, e)) { + if (EventFilter.acceptEvent(filters, e)) { fnAccepted(currentLine, e) } else { fnRejected(currentLine, e) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index b2d29e7c0ed6a..94698647eb5f3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -119,9 +119,9 @@ class FilteredEventLogFileRewriter( def rewrite(eventLogFiles: Seq[FileStatus]): String = { require(eventLogFiles.nonEmpty) - val targetEventLogFilePath = eventLogFiles.last.getPath - val logWriter = new CompactedEventLogFileWriter(targetEventLogFilePath, "dummy", None, - targetEventLogFilePath.getParent.toUri, sparkConf, hadoopConf) + val lastIndexEventLogPath = eventLogFiles.last.getPath + val logWriter = new CompactedEventLogFileWriter(lastIndexEventLogPath, "dummy", None, + lastIndexEventLogPath.getParent.toUri, sparkConf, hadoopConf) logWriter.start() eventLogFiles.foreach { file => diff --git a/docs/configuration.md b/docs/configuration.md index 41b8f513acde5..353a70be00cc5 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1036,9 +1036,9 @@ Apart from these, the following properties are also available, and may be useful will be applied across applications which are being loaded in Spark History Server, as well as compaction and cleanup would require running Spark History Server.
NOTE 2: Spark History Server may not compact the old event log files if it figures - out compaction on event log for such application won't reduce the size at expected - rate threshold. For streaming query (including Structured Streaming) we normally - expect compaction will run, but for batch query compaction won't run in most cases. + out not a lot of space would be reduced during compaction. For streaming query + (including Structured Streaming) we normally expect compaction will run, but for + batch query compaction won't run in most cases. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala index 4fdfb3cea0403..82213f34b84d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala @@ -89,14 +89,13 @@ private[spark] class SQLEventFilterBuilder extends SparkListener with EventFilte } private def onExecutionEnd(event: SparkListenerSQLExecutionEnd): Unit = { - val jobs = _liveExecutionToJobs.getOrElse(event.executionId, mutable.HashSet[Int]()) - _liveExecutionToJobs -= event.executionId - - val stagesToDrop = _jobToStages.filter(kv => jobs.contains(kv._1)).values.flatten - _jobToStages --= jobs - stages --= stagesToDrop - _stageToTasks --= stagesToDrop - _stageToRDDs --= stagesToDrop + _liveExecutionToJobs.remove(event.executionId).foreach { jobs => + val stagesToDrop = _jobToStages.filter(kv => jobs.contains(kv._1)).values.flatten + _jobToStages --= jobs + stages --= stagesToDrop + _stageToTasks --= stagesToDrop + _stageToRDDs --= stagesToDrop + } } override def createFilter(): EventFilter = { @@ -136,7 +135,7 @@ private[spark] class SQLLiveEntitiesEventFilter( case e: SparkListenerDriverAccumUpdates => liveExecutionToJobs.contains(e.executionId) - case e if acceptFnForJobEvents.isDefinedAt(e) && acceptFnForJobEvents(e) => + case e if acceptFnForJobEvents.lift(e).contains(true) => // NOTE: if acceptFnForJobEvents(e) returns false, we should leave it to "unmatched" true From 3286f9dfcea52378b79caf45f88cdebea4cca8d5 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 13 Dec 2019 13:06:38 +0900 Subject: [PATCH 24/28] Reflect review comments partially --- .../history/BasicEventFilterBuilder.scala | 11 +---- .../spark/deploy/history/EventFilter.scala | 46 +++++++----------- .../history/EventLogFileCompactor.scala | 47 ++++++++++++------- .../EventLogFilterRateCalculator.scala | 20 ++++---- .../history/EventLogFileCompactorSuite.scala | 42 ++++++++++++----- 5 files changed, 92 insertions(+), 74 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala index 74ff0e2e45ecf..dde7d4eb7bf61 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala @@ -82,15 +82,8 @@ private[spark] abstract class JobEventFilter( stageToTasks: Map[Int, Set[Long]], stageToRDDs: Map[Int, Seq[Int]]) extends EventFilter with Logging { - private val liveTasks: Set[Long] = stageToTasks.values match { - case xs if xs.isEmpty => Set.empty[Long] - case xs => xs.reduce(_ ++ _).toSet - } - - private val liveRDDs: Set[Int] = stageToRDDs.values match { - case xs if xs.isEmpty => Set.empty[Int] - case xs => xs.reduce(_ ++ _).toSet - } + private val liveTasks: Set[Long] = stageToTasks.values.flatten.toSet + private val liveRDDs: Set[Int] = stageToRDDs.values.flatten.toSet logDebug(s"jobs : ${jobToStages.keySet}") logDebug(s"stages in jobs : ${jobToStages.values.flatten}") diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala index 084ce7d849c6d..13ed3d6bca092 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala @@ -74,54 +74,42 @@ private[spark] trait EventFilter { } object EventFilter extends Logging { - def acceptEvent(filters: Seq[EventFilter], event: SparkListenerEvent): Boolean = { - val results = filters.flatMap(_.acceptFn().lift.apply(event)) - results.isEmpty || !results.contains(false) - } def applyFilterToFile( fs: FileSystem, filters: Seq[EventFilter], - path: Path)( - fnAccepted: (String, SparkListenerEvent) => Unit)( - fnRejected: (String, SparkListenerEvent) => Unit)( - fnUnidentified: String => Unit): Unit = { + path: Path, + onAccepted: (String, SparkListenerEvent) => Unit, + onRejected: (String, SparkListenerEvent) => Unit, + onUnidentified: String => Unit): Unit = { Utils.tryWithResource(EventLogFileReader.openEventLog(path, fs)) { in => val lines = Source.fromInputStream(in)(Codec.UTF8).getLines() - var currentLine: String = null - var lineNumber: Int = 0 - - try { - val lineEntries = lines.zipWithIndex - while (lineEntries.hasNext) { - val entry = lineEntries.next() - - currentLine = entry._1 - lineNumber = entry._2 + 1 - + lines.zipWithIndex.foreach { case (line, lineNum) => + try { val event = try { - Some(JsonProtocol.sparkEventFromJson(parse(currentLine))) + Some(JsonProtocol.sparkEventFromJson(parse(line))) } catch { // ignore any exception occurred from unidentified json case NonFatal(_) => - fnUnidentified(currentLine) + onUnidentified(line) None } event.foreach { e => - if (EventFilter.acceptEvent(filters, e)) { - fnAccepted(currentLine, e) + val results = filters.flatMap(_.acceptFn().lift.apply(e)) + if (results.isEmpty || !results.contains(false)) { + onAccepted(line, e) } else { - fnRejected(currentLine, e) + onRejected(line, e) } } + } catch { + case e: Exception => + logError(s"Exception parsing Spark event log: ${path.getName}", e) + logError(s"Malformed line #$lineNum: $line\n") + throw e } - } catch { - case e: Exception => - logError(s"Exception parsing Spark event log: ${path.getName}", e) - logError(s"Malformed line #$lineNumber: $currentLine\n") - throw e } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index 94698647eb5f3..94781c4a6e9c2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -38,14 +38,13 @@ import org.apache.spark.scheduler._ * to accept. * * This class assumes caller will provide the sorted list of files which are sorted by the index of - * event log file - caller should keep in mind that this class doesn't care about the semantic of - * ordering. + * event log file, with "at most" one compact file placed first if it exists. Caller should keep in + * mind that this class doesn't care about the semantic of ordering. * * When compacting the files, the range of compaction for given file list is determined as: - * (rightmost compact file ~ the file where there're `maxFilesToRetain` files on the right side) + * (first ~ the file where there're `maxFilesToRetain` files on the right side) * - * If there's no compact file in the list, it starts from the first file. If there're not enough - * files after rightmost compact file, compaction will be skipped. + * If there're not enough files on the range of compaction, compaction will be skipped. */ class EventLogFileCompactor( sparkConf: SparkConf, @@ -55,6 +54,8 @@ class EventLogFileCompactor( private val maxFilesToRetain: Int = sparkConf.get(EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN) def compact(eventLogFiles: Seq[FileStatus]): Seq[FileStatus] = { + assertPrecondition(eventLogFiles) + if (eventLogFiles.length <= maxFilesToRetain) { return eventLogFiles } @@ -79,6 +80,14 @@ class EventLogFileCompactor( } } + private def assertPrecondition(eventLogFiles: Seq[FileStatus]): Unit = { + val idxCompactedFiles = eventLogFiles.zipWithIndex.filter { case (file, _) => + EventLogFileWriter.isCompacted(file.getPath) + } + require(idxCompactedFiles.size < 2 && idxCompactedFiles.headOption.forall(_._2 == 0), + "The number of compact files should be at most 1, and should be placed first if exists.") + } + private def cleanupCompactedFiles(files: Seq[FileStatus]): Unit = { files.foreach { file => var deleted = false @@ -95,11 +104,19 @@ class EventLogFileCompactor( private def findFilesToCompact( eventLogFiles: Seq[FileStatus]): (Seq[FileStatus], Seq[FileStatus]) = { - val files = RollingEventLogFilesFileReader.dropBeforeLastCompactFile(eventLogFiles) - if (files.length > maxFilesToRetain) { - (files.dropRight(maxFilesToRetain), files.takeRight(maxFilesToRetain)) + val numNormalEventLogFiles = { + if (EventLogFileWriter.isCompacted(eventLogFiles.head.getPath)) { + eventLogFiles.length - 1 + } else { + eventLogFiles.length + } + } + + // This avoids compacting only compact file. + if (numNormalEventLogFiles > maxFilesToRetain) { + (eventLogFiles.dropRight(maxFilesToRetain), eventLogFiles.takeRight(maxFilesToRetain)) } else { - (Seq.empty, files) + (Seq.empty, eventLogFiles) } } } @@ -125,13 +142,11 @@ class FilteredEventLogFileRewriter( logWriter.start() eventLogFiles.foreach { file => - EventFilter.applyFilterToFile(fs, filters, file.getPath) { case (line, _) => - logWriter.writeEvent(line, flushLogger = true) - } { case (_, _) => - // no-op - } { line => - logWriter.writeEvent(line, flushLogger = true) - } + EventFilter.applyFilterToFile(fs, filters, file.getPath, + onAccepted = (line, _) => logWriter.writeEvent(line, flushLogger = true), + onRejected = (_, _) => {}, + onUnidentified = line => logWriter.writeEvent(line, flushLogger = true) + ) } logWriter.stop() diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculator.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculator.scala index db1ce3c0dcb86..07fd7cc3c43ee 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculator.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculator.scala @@ -51,15 +51,17 @@ class EventLogFilterRateCalculator(fs: FileSystem) { var filteredInEvents = 0L eventLogPaths.foreach { path => - EventFilter.applyFilterToFile(fs, filters, path) { case (_, _) => - allEvents += 1 - filteredInEvents += 1 - } { case (_, _) => - allEvents += 1 - } { _ => - allEvents += 1 - filteredInEvents += 1 - } + EventFilter.applyFilterToFile(fs, filters, path, + onAccepted = (_, _) => { + allEvents += 1 + filteredInEvents += 1 + }, + onRejected = (_, _) => allEvents += 1, + onUnidentified = _ => { + allEvents += 1 + filteredInEvents += 1 + } + ) } filteredInEvents.toDouble / allEvents diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala index 9f53976ea1319..ce470594156c3 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala @@ -77,21 +77,42 @@ class EventLogFileCompactorSuite extends SparkFunSuite { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - val logPaths = (1 to 5).map { idx => writeDummyEventLogFile(dir, idx) } + val logPaths = (1 to 2).map { idx => writeDummyEventLogFile(dir, idx) } - val fileToCompact = logPaths(2) + val fileToCompact = logPaths.head val compactedPath = new Path(fileToCompact + EventLogFileWriter.COMPACTED) assert(fs.rename(new Path(fileToCompact), compactedPath)) - val newLogPaths = logPaths.take(2) ++ Seq(compactedPath.toString) ++ - logPaths.takeRight(2) + val newLogPaths = Seq(compactedPath.toString) ++ logPaths.tail val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) val fileStatuses = newLogPaths.map { p => fs.getFileStatus(new Path(p)) } val filesToRead = compactor.compact(fileStatuses) - // filesToRead will start from rightmost compact file, but no new compact file - assert(filesToRead.map(_.getPath) === fileStatuses.takeRight(3).map(_.getPath)) + // there's no compaction + assert(filesToRead.map(_.getPath) === fileStatuses.map(_.getPath)) + } + } + + test("compact file exists, number of origin files are same as max files to retain") { + withTempDir { dir => + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + val logPaths = (1 to 4).map { idx => writeDummyEventLogFile(dir, idx) } + + val fileToCompact = logPaths.head + val compactedPath = new Path(fileToCompact + EventLogFileWriter.COMPACTED) + assert(fs.rename(new Path(fileToCompact), compactedPath)) + + // compact file in first idx, and same number of normal files as max files to retain + val newLogPaths = Seq(compactedPath.toString) ++ logPaths.tail + + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + val fileStatuses = newLogPaths.map { p => fs.getFileStatus(new Path(p)) } + val filesToRead = compactor.compact(fileStatuses) + + // no compaction will be performed + assert(filesToRead.map(_.getPath) === fileStatuses.map(_.getPath)) } } @@ -101,19 +122,18 @@ class EventLogFileCompactorSuite extends SparkFunSuite { val logPaths = (1 to 10).map { idx => writeDummyEventLogFile(dir, idx) } - val fileToCompact = logPaths(2) + val fileToCompact = logPaths.head val compactedPath = new Path(fileToCompact + EventLogFileWriter.COMPACTED) assert(fs.rename(new Path(fileToCompact), compactedPath)) - val newLogPaths = logPaths.take(2) ++ Seq(compactedPath.toString) ++ - logPaths.takeRight(7) + val newLogPaths = Seq(compactedPath.toString) ++ logPaths.tail val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) val fileStatuses = newLogPaths.map { p => fs.getFileStatus(new Path(p)) } assertCompaction(fs, fileStatuses, compactor.compact(fileStatuses)) - // compacted files will be removed - we don't check files "before" the rightmost compact file - fileStatuses.drop(2).dropRight(3).foreach { status => assert(!fs.exists(status.getPath)) } + // compacted files will be removed + fileStatuses.dropRight(3).foreach { status => assert(!fs.exists(status.getPath)) } } } From ab5d2332ce2490e2f3dcc6d5211dd708e02e35d9 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 13 Dec 2019 13:26:20 +0900 Subject: [PATCH 25/28] Add missed one --- .../apache/spark/deploy/history/EventLogFileReaders.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 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 301d44bfe042f..f0466b8ad8f55 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 @@ -228,7 +228,7 @@ class RollingEventLogFilesFileReader( idx } - val filesToRead = RollingEventLogFilesFileReader.dropBeforeLastCompactFile(eventLogFiles) + val filesToRead = dropBeforeLastCompactFile(eventLogFiles) val indices = filesToRead.map { file => getEventLogFileIndex(file.getPath.getName) } require((indices.head to indices.last) == indices, "Found missing event log file, expected" + s" indices: ${indices.head to indices.last}, actual: ${indices}") @@ -271,10 +271,8 @@ class RollingEventLogFilesFileReader( override def totalSize: Long = eventLogFiles.map(_.getLen).sum private def lastEventLogFile: FileStatus = eventLogFiles.last -} -object RollingEventLogFilesFileReader { - def dropBeforeLastCompactFile(eventLogFiles: Seq[FileStatus]): Seq[FileStatus] = { + private def dropBeforeLastCompactFile(eventLogFiles: Seq[FileStatus]): Seq[FileStatus] = { val lastCompactedFileIdx = eventLogFiles.lastIndexWhere { fs => EventLogFileWriter.isCompacted(fs.getPath) } From 27244f35150aaad89c51b438f9d6163729d798e4 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 16 Dec 2019 22:57:16 +0900 Subject: [PATCH 26/28] Inline FilterRateCalculator into EventLogFileCompactor, reflect changes, move compaction to the task of checkLogs() --- .../history/BasicEventFilterBuilder.scala | 22 +- .../spark/deploy/history/EventFilter.scala | 39 ++- .../history/EventLogFileCompactor.scala | 85 +++++-- .../deploy/history/EventLogFileWriters.scala | 5 +- .../EventLogFilterRateCalculator.scala | 70 ------ .../deploy/history/FsHistoryProvider.scala | 163 +++++++------ .../spark/internal/config/package.scala | 6 + .../history/BasicEventFilterSuite.scala | 15 +- .../history/EventLogFileCompactorSuite.scala | 196 +++++++++------- .../EventLogFilterRateCalculatorSuite.scala | 68 ------ .../deploy/history/EventLogTestHelper.scala | 37 ++- .../history/FsHistoryProviderSuite.scala | 222 +++++------------- .../history/SQLEventFilterBuilder.scala | 2 +- 13 files changed, 415 insertions(+), 515 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculator.scala delete mode 100644 core/src/test/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculatorSuite.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala index dde7d4eb7bf61..5dad210460fa1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.history import scala.collection.mutable +import org.apache.spark.deploy.history.EventFilter.FilterStatistic import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ @@ -33,12 +34,18 @@ private[spark] class BasicEventFilterBuilder extends SparkListener with EventFil private val _stageToRDDs = new mutable.HashMap[Int, Seq[Int]] private val _liveExecutors = new mutable.HashSet[String] + private var totalJobs: Long = 0L + private var totalStages: Long = 0L + private var totalTasks: Long = 0L + def liveJobToStages: Map[Int, Seq[Int]] = _liveJobToStages.toMap def stageToTasks: Map[Int, Set[Long]] = _stageToTasks.mapValues(_.toSet).toMap def stageToRDDs: Map[Int, Seq[Int]] = _stageToRDDs.toMap def liveExecutors: Set[String] = _liveExecutors.toSet override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + totalJobs += 1 + totalStages += jobStart.stageIds.length _liveJobToStages += jobStart.jobId -> jobStart.stageIds } @@ -55,6 +62,7 @@ private[spark] class BasicEventFilterBuilder extends SparkListener with EventFil } override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { + totalTasks += 1 val curTasks = _stageToTasks.getOrElseUpdate(taskStart.stageId, mutable.HashSet[Long]()) curTasks += taskStart.taskInfo.taskId @@ -69,6 +77,11 @@ private[spark] class BasicEventFilterBuilder extends SparkListener with EventFil } override def createFilter(): EventFilter = new BasicEventFilter(this) + + def statistic(): FilterStatistic = { + FilterStatistic(totalJobs, liveJobToStages.size, totalStages, + liveJobToStages.map(_._2.size).sum, totalTasks, _stageToTasks.map(_._2.size).sum) + } } /** @@ -78,6 +91,7 @@ private[spark] class BasicEventFilterBuilder extends SparkListener with EventFil * otherwise. */ private[spark] abstract class JobEventFilter( + stats: Option[FilterStatistic], jobToStages: Map[Int, Seq[Int]], stageToTasks: Map[Int, Set[Long]], stageToRDDs: Map[Int, Seq[Int]]) extends EventFilter with Logging { @@ -91,6 +105,8 @@ private[spark] abstract class JobEventFilter( logDebug(s"tasks in stages : ${stageToTasks.values.flatten}") logDebug(s"RDDs in stages : ${stageToRDDs.values.flatten}") + override def statistic(): Option[FilterStatistic] = stats + protected val acceptFnForJobEvents: PartialFunction[SparkListenerEvent, Boolean] = { case e: SparkListenerStageCompleted => stageToTasks.contains(e.stageInfo.stageId) @@ -132,14 +148,16 @@ private[spark] abstract class JobEventFilter( * will be considered as "Don't mind". */ private[spark] class BasicEventFilter( + _stats: FilterStatistic, _liveJobToStages: Map[Int, Seq[Int]], _stageToTasks: Map[Int, Set[Long]], _stageToRDDs: Map[Int, Seq[Int]], liveExecutors: Set[String]) - extends JobEventFilter(_liveJobToStages, _stageToTasks, _stageToRDDs) with Logging { + extends JobEventFilter(Some(_stats), _liveJobToStages, _stageToTasks, _stageToRDDs) with Logging { def this(builder: BasicEventFilterBuilder) = { - this(builder.liveJobToStages, builder.stageToTasks, builder.stageToRDDs, builder.liveExecutors) + this(builder.statistic(), builder.liveJobToStages, builder.stageToTasks, builder.stageToRDDs, + builder.liveExecutors) } logDebug(s"live executors : $liveExecutors") diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala index 13ed3d6bca092..c4fe59387e23f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala @@ -19,13 +19,13 @@ package org.apache.spark.deploy.history import java.util.ServiceLoader -import scala.collection.JavaConverters._ import scala.io.{Codec, Source} import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileSystem, Path} import org.json4s.jackson.JsonMethods.parse +import org.apache.spark.deploy.history.EventFilter.FilterStatistic import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ import org.apache.spark.util.{JsonProtocol, Utils} @@ -39,30 +39,18 @@ private[spark] trait EventFilterBuilder extends SparkListenerInterface { def createFilter(): EventFilter } -object EventFilterBuilder { +/** [[EventFilter]] decides whether the given event should be accepted or rejected. */ +private[spark] trait EventFilter { /** - * Loads all available EventFilterBuilders in classloader via ServiceLoader, and initializes - * them via replaying events in given files. + * Provide statistic information of event filter, which would be used for measuring the score + * of compaction. + * + * To simplify the condition, currently the fields of statistic are static, since major kinds of + * events compaction would filter out are job related event types. If the filter doesn't track + * with job related events, return None instead. */ - def initializeBuilders(fs: FileSystem, files: Seq[Path]): Seq[EventFilterBuilder] = { - val bus = new ReplayListenerBus() - - val builders = ServiceLoader.load(classOf[EventFilterBuilder], - Utils.getContextOrSparkClassLoader).asScala.toSeq - builders.foreach(bus.addListener) - - files.foreach { log => - Utils.tryWithResource(EventLogFileReader.openEventLog(log, fs)) { in => - bus.replay(in, log.getName) - } - } - - builders - } -} + def statistic(): Option[FilterStatistic] -/** [[EventFilter]] decides whether the given event should be accepted or rejected. */ -private[spark] trait EventFilter { /** * Classify whether the event is accepted or rejected by this filter. * @@ -74,6 +62,13 @@ private[spark] trait EventFilter { } object EventFilter extends Logging { + case class FilterStatistic( + totalJobs: Long, + liveJobs: Long, + totalStages: Long, + liveStages: Long, + totalTasks: Long, + liveTasks: Long) def applyFilterToFile( fs: FileSystem, diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index 94781c4a6e9c2..581d3335d3a03 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -19,14 +19,19 @@ package org.apache.spark.deploy.history import java.io.IOException import java.net.URI +import java.util.ServiceLoader + +import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.spark.SparkConf +import org.apache.spark.deploy.history.EventFilter.FilterStatistic import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN -import org.apache.spark.scheduler._ +import org.apache.spark.internal.config.{EVENT_LOG_COMPACTION_SCORE_THRESHOLD, EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN} +import org.apache.spark.scheduler.ReplayListenerBus +import org.apache.spark.util.Utils /** * This class compacts the old event log files into one compact file, via two phases reading: @@ -37,6 +42,10 @@ import org.apache.spark.scheduler._ * old event log files with filters. Rewrite the events to the compact file which the filters decide * to accept. * + * This class will calculate the score based on statistic from [[EventFilter]] instances, which + * represents approximate rate of filtered-out events. Score is being calculated via applying + * heuristic; task events tend to take most size in event log. + * * This class assumes caller will provide the sorted list of files which are sorted by the index of * event log file, with "at most" one compact file placed first if it exists. Caller should keep in * mind that this class doesn't care about the semantic of ordering. @@ -50,33 +59,34 @@ class EventLogFileCompactor( sparkConf: SparkConf, hadoopConf: Configuration, fs: FileSystem) extends Logging { - private val maxFilesToRetain: Int = sparkConf.get(EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN) + private val compactionThresholdScore: Double = sparkConf.get(EVENT_LOG_COMPACTION_SCORE_THRESHOLD) - def compact(eventLogFiles: Seq[FileStatus]): Seq[FileStatus] = { + def compact(eventLogFiles: Seq[FileStatus]): (CompactionResult.Value, Option[Long]) = { assertPrecondition(eventLogFiles) - if (eventLogFiles.length <= maxFilesToRetain) { - return eventLogFiles - } - - if (EventLogFileWriter.isCompacted(eventLogFiles.last.getPath)) { - return Seq(eventLogFiles.last) + if (eventLogFiles.length < maxFilesToRetain) { + return (CompactionResult.NOT_ENOUGH_FILES, None) } - val (filesToCompact, filesToRetain) = findFilesToCompact(eventLogFiles) + val filesToCompact = findFilesToCompact(eventLogFiles) if (filesToCompact.isEmpty) { - filesToRetain + (CompactionResult.NOT_ENOUGH_FILES, None) } else { - val builders = EventFilterBuilder.initializeBuilders(fs, filesToCompact.map(_.getPath)) + val builders = initializeBuilders(fs, filesToCompact.map(_.getPath)) - val rewriter = new FilteredEventLogFileRewriter(sparkConf, hadoopConf, fs, - builders.map(_.createFilter())) - val compactedPath = rewriter.rewrite(filesToCompact) + val filters = builders.map(_.createFilter()) + val minScore = filters.flatMap(_.statistic()).map(calculateScore).min - cleanupCompactedFiles(filesToCompact) - - fs.getFileStatus(new Path(compactedPath)) :: filesToRetain.toList + if (minScore < compactionThresholdScore) { + (CompactionResult.LOW_SCORE_FOR_COMPACTION, None) + } else { + val rewriter = new FilteredEventLogFileRewriter(sparkConf, hadoopConf, fs, filters) + rewriter.rewrite(filesToCompact) + cleanupCompactedFiles(filesToCompact) + (CompactionResult.SUCCESS, Some(RollingEventLogFilesWriter.getEventLogFileIndex( + filesToCompact.last.getPath.getName))) + } } } @@ -88,6 +98,33 @@ class EventLogFileCompactor( "The number of compact files should be at most 1, and should be placed first if exists.") } + /** + * Loads all available EventFilterBuilders in classloader via ServiceLoader, and initializes + * them via replaying events in given files. + */ + private def initializeBuilders(fs: FileSystem, files: Seq[Path]): Seq[EventFilterBuilder] = { + val bus = new ReplayListenerBus() + + val builders = ServiceLoader.load(classOf[EventFilterBuilder], + Utils.getContextOrSparkClassLoader).asScala.toSeq + builders.foreach(bus.addListener) + + files.foreach { log => + Utils.tryWithResource(EventLogFileReader.openEventLog(log, fs)) { in => + bus.replay(in, log.getName) + } + } + + builders + } + + private def calculateScore(stats: FilterStatistic): Double = { + // For now it's simply measuring how many task events will be filtered out (rejected) + // but it can be sophisticated later once we get more heuristic information and found + // the case where this simple calculation doesn't work. + (stats.totalTasks - stats.liveTasks) * 1.0 / stats.totalTasks + } + private def cleanupCompactedFiles(files: Seq[FileStatus]): Unit = { files.foreach { file => var deleted = false @@ -103,7 +140,7 @@ class EventLogFileCompactor( } private def findFilesToCompact( - eventLogFiles: Seq[FileStatus]): (Seq[FileStatus], Seq[FileStatus]) = { + eventLogFiles: Seq[FileStatus]): Seq[FileStatus] = { val numNormalEventLogFiles = { if (EventLogFileWriter.isCompacted(eventLogFiles.head.getPath)) { eventLogFiles.length - 1 @@ -114,13 +151,17 @@ class EventLogFileCompactor( // This avoids compacting only compact file. if (numNormalEventLogFiles > maxFilesToRetain) { - (eventLogFiles.dropRight(maxFilesToRetain), eventLogFiles.takeRight(maxFilesToRetain)) + eventLogFiles.dropRight(maxFilesToRetain) } else { - (Seq.empty, eventLogFiles) + Seq.empty } } } +object CompactionResult extends Enumeration { + val SUCCESS, NOT_ENOUGH_FILES, LOW_SCORE_FOR_COMPACTION = Value +} + /** * This class rewrites the event log files into one compact file: the compact file will only * contain the events which pass the filters. Events will be dropped only when all filters 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 3111d303653b1..1d58d054b7825 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 @@ -214,7 +214,7 @@ class SingleEventLogFileWriter( override val logPath: String = SingleEventLogFileWriter.getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) - private val inProgressPath = logPath + EventLogFileWriter.IN_PROGRESS + protected def inProgressPath = logPath + EventLogFileWriter.IN_PROGRESS override def start(): Unit = { requireLogBaseDirAsDirectory() @@ -333,7 +333,8 @@ class RollingEventLogFilesWriter( writeLine(eventJson, flushLogger) } - private def rollEventLogFile(): Unit = { + /** exposed for testing only */ + private[history] def rollEventLogFile(): Unit = { closeWriter() index += 1 diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculator.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculator.scala deleted file mode 100644 index 07fd7cc3c43ee..0000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculator.scala +++ /dev/null @@ -1,70 +0,0 @@ -/* - * 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.{FileSystem, Path} - -import org.apache.spark.scheduler.SparkListenerEvent - -/** - * This class calculates the rate of events being accepted, via two phases reading: - * - * 1) Initialize available [[EventFilterBuilder]] instances, and replay the event log files with - * builders, so that these builders can gather the information to create [[EventFilter]] instances. - * 2) Initialize [[EventFilter]] instances from [[EventFilterBuilder]] instances, and replay the - * event log files with filters. Counts the number of events vs accepted events and calculate - * the rate. - */ -class EventLogFilterRateCalculator(fs: FileSystem) { - def calculate(eventLogPaths: Seq[Path]): Double = { - val builders = EventFilterBuilder.initializeBuilders(fs, eventLogPaths) - doCalculate(eventLogPaths, builders.map(_.createFilter())) - } - - /** - * Exposed for tests - enable UTs to simply inject EventFilters instead of loading from - * ServiceLoader which is subject to change on which modules are available on classloader. - */ - private[spark] def doCalculate(eventLogPaths: Seq[Path], filters: Seq[EventFilter]): Double = { - val calc = new Calculator(fs, filters) - calc.calculate(eventLogPaths) - } - - private class Calculator(fs: FileSystem, filters: Seq[EventFilter]) { - def calculate(eventLogPaths: Seq[Path]): Double = { - var allEvents = 0L - var filteredInEvents = 0L - - eventLogPaths.foreach { path => - EventFilter.applyFilterToFile(fs, filters, path, - onAccepted = (_, _) => { - allEvents += 1 - filteredInEvents += 1 - }, - onRejected = (_, _) => allEvents += 1, - onUnidentified = _ => { - allEvents += 1 - filteredInEvents += 1 - } - ) - } - - filteredInEvents.toDouble / allEvents - } - } -} 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 652b8bfc1d35a..fc6619b28281b 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 @@ -162,8 +162,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private val fileCompactor = new EventLogFileCompactor(conf, hadoopConf, fs) - private val eventFilterRateCalculator = new EventLogFilterRateCalculator(fs) - private val blacklist = new ConcurrentHashMap[String, Long] // Visible for testing @@ -476,7 +474,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) attempt.logPath, attempt.fileSize, attempt.lastIndex, - attempt.compactible, attempt.adminAcls, attempt.viewAcls, attempt.adminAclsGroups, @@ -519,8 +516,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val tasks = updated.flatMap { entry => try { val task: Future[Unit] = replayExecutor.submit(() => { - val compactible = getOrUpdateCompactible(entry) - mergeApplicationListing(entry, newLastScanTime, true, compactible) + val updatedLastCompactionIndex = compact(entry) + mergeApplicationListing(entry, newLastScanTime, true, updatedLastCompactionIndex) }) Some(task -> entry.rootPath) } catch { @@ -582,6 +579,30 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } + /** exposed for testing */ + private[history] def compact(reader: EventLogFileReader): Option[Long] = { + reader.lastIndex match { + case Some(lastIndex) => + try { + val info = listing.read(classOf[LogInfo], reader.rootPath.toString) + if (info.lastCompactionIndex.isEmpty || info.lastCompactionIndex.get < lastIndex) { + // haven't tried compaction for this index, do compaction + val (_, lastCompactionIndex) = fileCompactor.compact(reader.listEventLogFiles) + listing.write(info.copy(lastCompactionIndex = lastCompactionIndex)) + Some(lastIndex) + } else { + info.lastCompactionIndex + } + } catch { + case _: NoSuchElementException => + // this should exist, but ignoring doesn't hurt much + None + } + + case None => None // This is not applied to single event log file. + } + } + private[history] def shouldReloadLog(info: LogInfo, reader: EventLogFileReader): Boolean = { if (info.isComplete != reader.completed) { true @@ -670,41 +691,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } - private[spark] def getOrUpdateCompactible(reader: EventLogFileReader): Option[Boolean] = { - try { - val info = listing.read(classOf[LogInfo], reader.rootPath.toString) - val compactible = checkEligibilityForCompaction(info, reader) - if (info.compactible != compactible) { - listing.write(info.copy(compactible = compactible)) - } - compactible - } catch { - case _: NoSuchElementException => None - } - } - - protected def checkEligibilityForCompaction( - info: LogInfo, - reader: EventLogFileReader): Option[Boolean] = { - info.compactible.orElse { - // This is not applied to single event log file. - if (reader.lastIndex.isEmpty) { - Some(false) - } else { - if (reader.listEventLogFiles.length > 1) { - // We have at least one 'complete' file to check whether the event log is eligible to - // compact further. - val rate = eventFilterRateCalculator.calculate( - Seq(reader.listEventLogFiles.head.getPath)) - Some(rate < COMPACT_THRESHOLD_FILTER_IN_RATE) - } else { - // still cannot decide - None - } - } - } - } - /** * Replay the given log file, saving the application in the listing db. */ @@ -712,7 +698,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) reader: EventLogFileReader, scanTime: Long, enableOptimizations: Boolean, - compactible: Option[Boolean]): Unit = { + lastCompactionIndex: Option[Long]): Unit = { val eventsFilter: ReplayEventsFilter = { eventString => eventString.startsWith(APPL_START_EVENT_PREFIX) || eventString.startsWith(APPL_END_EVENT_PREFIX) || @@ -731,7 +717,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) ((!appCompleted && fastInProgressParsing) || reparseChunkSize > 0) val bus = new ReplayListenerBus() - val listener = new AppListingListener(reader, compactible, clock, shouldHalt) + val listener = new AppListingListener(reader, clock, shouldHalt) bus.addListener(listener) logInfo(s"Parsing $logPath for listing data...") @@ -791,7 +777,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) addListing(app) listing.write(LogInfo(logPath.toString(), scanTime, LogType.EventLogs, Some(app.info.id), app.attempts.head.info.attemptId, reader.fileSizeForLastIndex, reader.lastIndex, - app.attempts.head.compactible, reader.completed)) + lastCompactionIndex, reader.completed)) // For a finished log, remove the corresponding "in progress" entry from the listing DB if // the file is really gone. @@ -815,7 +801,8 @@ 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(reader, scanTime, enableOptimizations = false, compactible) + mergeApplicationListing(reader, scanTime, enableOptimizations = false, + lastCompactionIndex) case _ => // If the app hasn't written down its app ID to the logs, still record the entry in the @@ -823,7 +810,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // does not make progress after the configured max log age. listing.write( LogInfo(logPath.toString(), scanTime, LogType.EventLogs, None, None, - reader.fileSizeForLastIndex, reader.lastIndex, compactible, reader.completed)) + reader.fileSizeForLastIndex, reader.lastIndex, lastCompactionIndex, reader.completed)) } } @@ -990,8 +977,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private[spark] def rebuildAppStore( store: KVStore, reader: EventLogFileReader, - lastUpdated: Long, - compactible: Option[Boolean]): Unit = { + 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. val replayConf = conf.clone().set(ASYNC_TRACKING_ENABLED, false) @@ -1008,14 +994,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) try { val eventLogFiles = reader.listEventLogFiles - val newEventLogFiles = if (compactible.contains(true)) { - logInfo(s"Compacting ${reader.rootPath}...") - fileCompactor.compact(eventLogFiles) - } else { - eventLogFiles - } logInfo(s"Parsing ${reader.rootPath} to re-build UI...") - parseAppEventLogs(newEventLogFiles, replayBus, !reader.completed) + parseAppEventLogs(eventLogFiles, replayBus, !reader.completed) trackingStore.close(false) logInfo(s"Finished parsing ${reader.rootPath}") } catch { @@ -1126,30 +1106,61 @@ 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 reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), - 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) - val newStorePath = try { - Utils.tryWithResource(KVUtils.open(lease.tmpPath, metadata)) { store => - rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime(), attempt.compactible) + var retried = false + var newStorePath: File = null + + while(newStorePath == null) { + val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), + 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) + try { + Utils.tryWithResource(KVUtils.open(lease.tmpPath, metadata)) { store => + rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime()) + } + newStorePath = lease.commit(appId, attempt.info.attemptId) + } catch { + case _: IOException if !retried => + // compaction may touch the file(s) which app rebuild wants to read + // compaction wouldn't run in short interval, so try again... + logWarning(s"Exception occurred while rebuilding app $appId - trying again...") + lease.rollback() + retried = true + + case e: Exception => + lease.rollback() + throw e } - lease.commit(appId, attempt.info.attemptId) - } catch { - case e: Exception => - lease.rollback() - throw e } KVUtils.open(newStorePath, metadata) } private def createInMemoryStore(attempt: AttemptInfoWrapper): KVStore = { - val store = new InMemoryStore() - val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), - attempt.lastIndex) - rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime(), attempt.compactible) + var retried = false + var store: KVStore = null + + while (store == null) { + try { + val s = new InMemoryStore() + val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), + attempt.lastIndex) + rebuildAppStore(s, reader, attempt.info.lastUpdated.getTime()) + store = s + } catch { + case _: IOException if !retried => + // compaction may touch the file(s) which app rebuild wants to read + // compaction wouldn't run in short interval, so try again... + logWarning(s"Exception occurred while rebuilding log path ${attempt.logPath} - " + + "trying again...") + retried = true + + case e: Exception => + throw e + } + } + store } @@ -1224,7 +1235,7 @@ private[history] case class LogInfo( fileSize: Long, @JsonDeserialize(contentAs = classOf[JLong]) lastIndex: Option[Long], - compactible: Option[Boolean], + lastCompactionIndex: Option[Long], isComplete: Boolean) private[history] class AttemptInfoWrapper( @@ -1233,7 +1244,6 @@ private[history] class AttemptInfoWrapper( val fileSize: Long, @JsonDeserialize(contentAs = classOf[JLong]) val lastIndex: Option[Long], - val compactible: Option[Boolean], val adminAcls: Option[String], val viewAcls: Option[String], val adminAclsGroups: Option[String], @@ -1258,13 +1268,12 @@ private[history] class ApplicationInfoWrapper( private[history] class AppListingListener( reader: EventLogFileReader, - compactible: Option[Boolean], clock: Clock, haltEnabled: Boolean) extends SparkListener { private val app = new MutableApplicationInfo() private val attempt = new MutableAttemptInfo(reader.rootPath.getName(), - reader.fileSizeForLastIndex, reader.lastIndex, compactible) + reader.fileSizeForLastIndex, reader.lastIndex) private var gotEnvUpdate = false private var halted = false @@ -1352,8 +1361,7 @@ private[history] class AppListingListener( private class MutableAttemptInfo( logPath: String, fileSize: Long, - lastIndex: Option[Long], - compactible: Option[Boolean]) { + lastIndex: Option[Long]) { var attemptId: Option[String] = None var startTime = new Date(-1) var endTime = new Date(-1) @@ -1383,7 +1391,6 @@ private[history] class AppListingListener( logPath, fileSize, lastIndex, - compactible, 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 cea7bc1bbdef6..20163c227d5a1 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 @@ -204,6 +204,12 @@ package object config { .checkValue(_ > 0, "Max event log files to retain should be higher than 0.") .createWithDefault(Integer.MAX_VALUE) + private[spark] val EVENT_LOG_COMPACTION_SCORE_THRESHOLD = + ConfigBuilder("spark.eventLog.rolling.compaction.score.threshold") + .internal() + .doubleConf + .createWithDefault(0.7d) + private[spark] val EXECUTOR_ID = ConfigBuilder("spark.executor.id").stringConf.createOptional diff --git a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala index 0f742fb57ab96..18fa2b52004ea 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.history import org.apache.spark.{storage, SparkFunSuite, Success, TaskState} +import org.apache.spark.deploy.history.EventFilter.FilterStatistic import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.scheduler._ import org.apache.spark.status.ListenerEventsTestHelper @@ -25,6 +26,7 @@ import org.apache.spark.storage.{BlockManagerId, RDDBlockId, StorageLevel} class BasicEventFilterSuite extends SparkFunSuite { import ListenerEventsTestHelper._ + import BasicEventFilterSuite._ test("filter out events for finished jobs") { // assume finished job 1 with stage 1, tasks (1, 2), rdds (1, 2) @@ -33,8 +35,10 @@ class BasicEventFilterSuite extends SparkFunSuite { val stageToTasks: Map[Int, Set[Long]] = Map(2 -> Set(3, 4), 3 -> Set(5, 6)) val stageToRDDs: Map[Int, Seq[Int]] = Map(2 -> Seq(3, 4), 3 -> Seq(5, 6)) val liveExecutors: Set[String] = Set("1", "2") + val filterStats = FilterStatistic(2, 1, 2, 1, 4, 2) - val filter = new BasicEventFilter(liveJobToStages, stageToTasks, stageToRDDs, liveExecutors) + val filter = new BasicEventFilter(filterStats, liveJobToStages, stageToTasks, stageToRDDs, + liveExecutors) val acceptFn = filter.acceptFn().lift // Verifying with finished job 1 @@ -96,7 +100,8 @@ class BasicEventFilterSuite extends SparkFunSuite { // assume executor 1 was dead, and live executor 2 is available val liveExecutors: Set[String] = Set("2") - val filter = new BasicEventFilter(Map.empty, Map.empty, Map.empty, liveExecutors) + val filter = new BasicEventFilter(EMPTY_STATS, Map.empty, Map.empty, Map.empty, + liveExecutors) val acceptFn = filter.acceptFn().lift // events for dead executor should be rejected @@ -129,7 +134,7 @@ class BasicEventFilterSuite extends SparkFunSuite { assert(predicate === None) } - val filter = new BasicEventFilter(Map.empty, Map.empty, Map.empty, Set.empty) + val filter = new BasicEventFilter(EMPTY_STATS, Map.empty, Map.empty, Map.empty, Set.empty) val acceptFn = filter.acceptFn().lift assertNone(acceptFn(SparkListenerEnvironmentUpdate(Map.empty))) @@ -187,3 +192,7 @@ class BasicEventFilterSuite extends SparkFunSuite { } } } + +object BasicEventFilterSuite { + val EMPTY_STATS = FilterStatistic(0, 0, 0, 0, 0, 0) +} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala index ce470594156c3..882a5f183ee5b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala @@ -24,14 +24,19 @@ import scala.io.{Codec, Source} import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.json4s.jackson.JsonMethods.parse -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkFunSuite, Success} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.config.EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN +import org.apache.spark.deploy.history.EventLogTestHelper.writeEventsToRollingWriter +import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.internal.config.{EVENT_LOG_COMPACTION_SCORE_THRESHOLD, EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.status.ListenerEventsTestHelper import org.apache.spark.util.{JsonProtocol, Utils} class EventLogFileCompactorSuite extends SparkFunSuite { + import ListenerEventsTestHelper._ + private val sparkConf = testSparkConf() private val hadoopConf = SparkHadoopUtil.newConfiguration(sparkConf) @@ -39,7 +44,9 @@ class EventLogFileCompactorSuite extends SparkFunSuite { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) - assert(Seq.empty[FileStatus] === compactor.compact(Seq.empty)) + + assertNoCompaction(fs, Seq.empty, compactor.compact(Seq.empty), + CompactionResult.NOT_ENOUGH_FILES) } } @@ -47,14 +54,11 @@ class EventLogFileCompactorSuite extends SparkFunSuite { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - val logPath1 = writeDummyEventLogFile(dir, 1) - val logPath2 = writeDummyEventLogFile(dir, 2) - + val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + (1 to 2).map(_ => testEvent): _*) val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) - val fileStatuses = Seq(logPath1, logPath2).map { p => fs.getFileStatus(new Path(p)) } - val filesToRead = compactor.compact(fileStatuses) - - assert(filesToRead.map(_.getPath) === fileStatuses.map(_.getPath)) + assertNoCompaction(fs, fileStatuses, compactor.compact(fileStatuses), + CompactionResult.NOT_ENOUGH_FILES) } } @@ -62,14 +66,11 @@ class EventLogFileCompactorSuite extends SparkFunSuite { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - val logPaths = (1 to 5).map { idx => writeDummyEventLogFile(dir, idx) } - + val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + (1 to 5).map(_ => testEvent): _*) val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) - val fileStatuses = logPaths.map { p => fs.getFileStatus(new Path(p)) } - assertCompaction(fs, fileStatuses, compactor.compact(fileStatuses)) - - // compacted files will be removed - fileStatuses.take(2).foreach { status => assert(!fs.exists(status.getPath)) } + assertCompaction(fs, fileStatuses, compactor.compact(fileStatuses), + expectedNumOfFilesCompacted = 2) } } @@ -77,20 +78,18 @@ class EventLogFileCompactorSuite extends SparkFunSuite { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - val logPaths = (1 to 2).map { idx => writeDummyEventLogFile(dir, idx) } - - val fileToCompact = logPaths.head - val compactedPath = new Path(fileToCompact + EventLogFileWriter.COMPACTED) - assert(fs.rename(new Path(fileToCompact), compactedPath)) + val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + (1 to 2).map(_ => testEvent): _*) - val newLogPaths = Seq(compactedPath.toString) ++ logPaths.tail + val fileToCompact = fileStatuses.head.getPath + val compactedPath = new Path(fileToCompact.getParent, + fileToCompact.getName + EventLogFileWriter.COMPACTED) + assert(fs.rename(fileToCompact, compactedPath)) + val newFileStatuses = Seq(fs.getFileStatus(compactedPath)) ++ fileStatuses.drop(1) val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) - val fileStatuses = newLogPaths.map { p => fs.getFileStatus(new Path(p)) } - val filesToRead = compactor.compact(fileStatuses) - - // there's no compaction - assert(filesToRead.map(_.getPath) === fileStatuses.map(_.getPath)) + assertNoCompaction(fs, newFileStatuses, compactor.compact(newFileStatuses), + CompactionResult.NOT_ENOUGH_FILES) } } @@ -98,21 +97,18 @@ class EventLogFileCompactorSuite extends SparkFunSuite { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - val logPaths = (1 to 4).map { idx => writeDummyEventLogFile(dir, idx) } - - val fileToCompact = logPaths.head - val compactedPath = new Path(fileToCompact + EventLogFileWriter.COMPACTED) - assert(fs.rename(new Path(fileToCompact), compactedPath)) + val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + (1 to 4).map(_ => testEvent): _*) - // compact file in first idx, and same number of normal files as max files to retain - val newLogPaths = Seq(compactedPath.toString) ++ logPaths.tail + val fileToCompact = fileStatuses.head.getPath + val compactedPath = new Path(fileToCompact.getParent, + fileToCompact.getName + EventLogFileWriter.COMPACTED) + assert(fs.rename(fileToCompact, compactedPath)) + val newFileStatuses = Seq(fs.getFileStatus(compactedPath)) ++ fileStatuses.drop(1) val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) - val fileStatuses = newLogPaths.map { p => fs.getFileStatus(new Path(p)) } - val filesToRead = compactor.compact(fileStatuses) - - // no compaction will be performed - assert(filesToRead.map(_.getPath) === fileStatuses.map(_.getPath)) + assertNoCompaction(fs, newFileStatuses, compactor.compact(newFileStatuses), + CompactionResult.NOT_ENOUGH_FILES) } } @@ -120,20 +116,18 @@ class EventLogFileCompactorSuite extends SparkFunSuite { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - val logPaths = (1 to 10).map { idx => writeDummyEventLogFile(dir, idx) } + val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + (1 to 10).map(_ => testEvent): _*) - val fileToCompact = logPaths.head - val compactedPath = new Path(fileToCompact + EventLogFileWriter.COMPACTED) - assert(fs.rename(new Path(fileToCompact), compactedPath)) - - val newLogPaths = Seq(compactedPath.toString) ++ logPaths.tail + val fileToCompact = fileStatuses.head.getPath + val compactedPath = new Path(fileToCompact.getParent, + fileToCompact.getName + EventLogFileWriter.COMPACTED) + assert(fs.rename(fileToCompact, compactedPath)) + val newFileStatuses = Seq(fs.getFileStatus(compactedPath)) ++ fileStatuses.drop(1) val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) - val fileStatuses = newLogPaths.map { p => fs.getFileStatus(new Path(p)) } - assertCompaction(fs, fileStatuses, compactor.compact(fileStatuses)) - - // compacted files will be removed - fileStatuses.dropRight(3).foreach { status => assert(!fs.exists(status.getPath)) } + assertCompaction(fs, newFileStatuses, compactor.compact(newFileStatuses), + expectedNumOfFilesCompacted = 7) } } @@ -142,24 +136,23 @@ class EventLogFileCompactorSuite extends SparkFunSuite { val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) // 1, 2 will be compacted into one file, 3~5 are dummies to ensure max files to retain - val logPath1 = EventLogTestHelper.writeEventLogFile(sparkConf, hadoopConf, dir, 1, Seq( - SparkListenerExecutorAdded(0, "exec1", new ExecutorInfo("host1", 1, Map.empty)), - SparkListenerJobStart(1, 0, Seq.empty))) - val logPath2 = EventLogTestHelper.writeEventLogFile(sparkConf, hadoopConf, dir, 2, Seq( - SparkListenerJobEnd(1, 1, JobSucceeded), - SparkListenerExecutorAdded(2, "exec2", new ExecutorInfo("host2", 1, Map.empty)))) - val logPaths = Seq(logPath1, logPath2) ++ (3 to 5).map { idx => - writeDummyEventLogFile(dir, idx) - } + val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + Seq( + SparkListenerExecutorAdded(0, "exec1", new ExecutorInfo("host1", 1, Map.empty)), + SparkListenerJobStart(1, 0, Seq.empty)), + Seq( + SparkListenerJobEnd(1, 1, JobSucceeded), + SparkListenerExecutorAdded(2, "exec2", new ExecutorInfo("host2", 1, Map.empty))), + testEvent, + testEvent, + testEvent) val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) - val fileStatuses = logPaths.map { p => fs.getFileStatus(new Path(p)) } - val filesToRead = compactor.compact(fileStatuses) - - // 3 (max file to retain) + 1 (compacted file) - assert(filesToRead.length === 4) - val compactFilePath = filesToRead.head.getPath + assertCompaction(fs, fileStatuses, compactor.compact(fileStatuses), + expectedNumOfFilesCompacted = 2) + val expectCompactFileBasePath = fileStatuses.take(2).last.getPath + val compactFilePath = getCompactFilePath(expectCompactFileBasePath) Utils.tryWithResource(EventLogFileReader.openEventLog(compactFilePath, fs)) { is => val lines = Source.fromInputStream(is)(Codec.UTF8).getLines().toList assert(lines.length === 2, "Compacted file should have only two events being accepted") @@ -172,28 +165,73 @@ class EventLogFileCompactorSuite extends SparkFunSuite { } } + test("Don't compact file if score is lower than threshold") { + withTempDir { dir => + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + val newConf = sparkConf.set(EVENT_LOG_COMPACTION_SCORE_THRESHOLD, 0.7d) + + // only one of two tasks is finished, which would score 0.5d + val tasks = createTasks(2, Array("exec1"), 0L).map(createTaskStartEvent(_, 1, 0)) + + val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, newConf, hadoopConf, + tasks, + Seq(SparkListenerTaskEnd(1, 0, "taskType", Success, tasks.head.taskInfo, + new ExecutorMetrics, null)), + testEvent, + testEvent, + testEvent) + + val compactor = new EventLogFileCompactor(newConf, hadoopConf, fs) + assertNoCompaction(fs, fileStatuses, compactor.compact(fileStatuses), + CompactionResult.LOW_SCORE_FOR_COMPACTION) + } + } + private def assertCompaction( fs: FileSystem, originalFiles: Seq[FileStatus], - retValue: Seq[FileStatus]): Unit = { - // 3 (max file to retain) + 1 (compacted file) - assert(retValue.length === 4) - val originalFilesToRead = retValue.takeRight(3) - val originFileToCompact = originalFiles.takeRight(4).head.getPath - val compactFilePath = retValue.head.getPath + compactRet: (CompactionResult.Value, Option[Long]), + expectedNumOfFilesCompacted: Int): Unit = { + assert(CompactionResult.SUCCESS === compactRet._1) + + val expectRetainedFiles = originalFiles.drop(expectedNumOfFilesCompacted) + expectRetainedFiles.foreach { status => assert(fs.exists(status.getPath)) } - assert(compactFilePath.getName === originFileToCompact.getName + EventLogFileWriter.COMPACTED) - assert(originalFilesToRead.map(_.getPath) === originalFiles.takeRight(3).map(_.getPath)) + val expectRemovedFiles = originalFiles.take(expectedNumOfFilesCompacted) + expectRemovedFiles.foreach { status => assert(!fs.exists(status.getPath)) } - retValue.foreach { status => assert(fs.exists(status.getPath)) } + val expectCompactFileBasePath = originalFiles.take(expectedNumOfFilesCompacted).last.getPath + val expectCompactFileIndex = RollingEventLogFilesWriter.getEventLogFileIndex( + expectCompactFileBasePath.getName) + assert(Some(expectCompactFileIndex) === compactRet._2) + + val expectCompactFilePath = getCompactFilePath(expectCompactFileBasePath) + assert(fs.exists(expectCompactFilePath)) + } + + private def getCompactFilePath(expectCompactFileBasePath: Path): Path = { + new Path(expectCompactFileBasePath.getParent, + expectCompactFileBasePath.getName + EventLogFileWriter.COMPACTED) } - private def writeDummyEventLogFile(dir: File, idx: Int): String = { - EventLogTestHelper.writeEventLogFile(sparkConf, hadoopConf, dir, idx, - Seq(SparkListenerApplicationStart("app", Some("app"), 0, "user", None))) + private def assertNoCompaction( + fs: FileSystem, + originalFiles: Seq[FileStatus], + compactRet: (CompactionResult.Value, Option[Long]), + expectedCompactRet: CompactionResult.Value): Unit = { + assert(compactRet._1 === expectedCompactRet) + assert(None === compactRet._2) + originalFiles.foreach { status => assert(fs.exists(status.getPath)) } } + private def testEvent: Seq[SparkListenerEvent] = + Seq(SparkListenerApplicationStart("app", Some("app"), 0, "user", None)) + private def testSparkConf(): SparkConf = { - new SparkConf().set(EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN, 3) + new SparkConf() + .set(EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN, 3) + // to simplify the tests, we set the score threshold as 0.0d + // individual test can override the value to verify the functionality + .set(EVENT_LOG_COMPACTION_SCORE_THRESHOLD, 0.0d) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculatorSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculatorSuite.scala deleted file mode 100644 index a75f3ae0ba488..0000000000000 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFilterRateCalculatorSuite.scala +++ /dev/null @@ -1,68 +0,0 @@ -/* - * 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 scala.collection.mutable - -import org.apache.hadoop.fs.Path - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.history.EventLogTestHelper.{TestEventFilter1, TestEventFilter2} -import org.apache.spark.scheduler.{SparkListenerApplicationEnd, SparkListenerApplicationStart, SparkListenerBlockManagerAdded, SparkListenerEvent, SparkListenerExecutorAdded, SparkListenerJobStart, SparkListenerNodeBlacklisted, SparkListenerNodeUnblacklisted, SparkListenerTaskStart, SparkListenerUnpersistRDD} -import org.apache.spark.storage.BlockManagerId - -class EventLogFilterRateCalculatorSuite extends SparkFunSuite { - private val sparkConf = new SparkConf() - private val hadoopConf = SparkHadoopUtil.newConfiguration(sparkConf) - - test("calculate accept rate") { - withTempDir { dir => - val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - - val events = new mutable.ArrayBuffer[SparkListenerEvent] - - // filterApplicationEnd: Some(true) & Some(true) => accepted - events += SparkListenerApplicationEnd(0) - - // filterBlockManagerAdded: Some(true) & Some(false) => rejected - events += SparkListenerBlockManagerAdded(0, BlockManagerId("1", "host1", 1), 10) - - // filterApplicationStart: Some(false) & Some(false) => rejected - events += SparkListenerApplicationStart("app", None, 0, "user", None) - - // filterNodeBlacklisted: None & Some(true) => accepted - events += SparkListenerNodeBlacklisted(0, "host1", 1) - - // filterNodeUnblacklisted: None & Some(false) => rejected - events += SparkListenerNodeUnblacklisted(0, "host1") - - // other events: None & None => accepted - events += SparkListenerUnpersistRDD(0) - - val logPath = EventLogTestHelper.writeEventLogFile(sparkConf, hadoopConf, dir, 1, events) - val logPath2 = EventLogTestHelper.writeEventLogFile(sparkConf, hadoopConf, dir, 2, events) - val logs = Seq(logPath, logPath2).map(new Path(_)) - - val filters = Seq(new TestEventFilter1, new TestEventFilter2) - val calculator = new EventLogFilterRateCalculator(fs) - // 6 accepted, 6 rejected - assert(0.5d === calculator.doCalculate(logs, filters)) - } - } -} 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 0ac8fad53bdd9..66e3fb788939e 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 @@ -21,12 +21,12 @@ import java.io.File import java.nio.charset.StandardCharsets import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.json4s.jackson.JsonMethods.{compact, render} import org.apache.spark.SparkConf import org.apache.spark.internal.config._ -import org.apache.spark.scheduler.{SparkListenerApplicationEnd, SparkListenerApplicationStart, SparkListenerBlockManagerAdded, SparkListenerEnvironmentUpdate, SparkListenerEvent, SparkListenerNodeBlacklisted, SparkListenerNodeUnblacklisted} +import org.apache.spark.scheduler._ import org.apache.spark.util.JsonProtocol object EventLogTestHelper { @@ -77,6 +77,35 @@ object EventLogTestHelper { writer.logPath } + def writeEventsToRollingWriter( + fs: FileSystem, + appId: String, + dir: File, + sparkConf: SparkConf, + hadoopConf: Configuration, + eventsFiles: Seq[SparkListenerEvent]*): Seq[FileStatus] = { + val writer = new RollingEventLogFilesWriter(appId, None, dir.toURI, sparkConf, hadoopConf) + writer.start() + + eventsFiles.dropRight(1).foreach { events => + writeEventsToRollingWriter(writer, events, rollFile = true) + } + eventsFiles.lastOption.foreach { events => + writeEventsToRollingWriter(writer, events, rollFile = false) + } + + writer.stop() + EventLogFileReader(fs, new Path(writer.logPath)).get.listEventLogFiles + } + + def writeEventsToRollingWriter( + writer: RollingEventLogFilesWriter, + events: Seq[SparkListenerEvent], + rollFile: Boolean): Unit = { + events.foreach { event => writer.writeEvent(convertEvent(event), flushLogger = true) } + if (rollFile) writer.rollEventLogFile() + } + def convertEvent(event: SparkListenerEvent): String = { compact(render(JsonProtocol.sparkEventToJson(event))) } @@ -87,6 +116,8 @@ object EventLogTestHelper { case _: SparkListenerBlockManagerAdded => true case _: SparkListenerApplicationStart => false } + + override def statistic(): Option[EventFilter.FilterStatistic] = None } class TestEventFilter2 extends EventFilter { @@ -98,5 +129,7 @@ object EventLogTestHelper { case _: SparkListenerApplicationStart => false case _: SparkListenerNodeUnblacklisted => false } + + override def statistic(): Option[EventFilter.FilterStatistic] = None } } 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 b46140b92a67e..c27405e0c89cd 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 @@ -39,8 +39,9 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.{JobExecutionStatus, SecurityManager, SPARK_VERSION, SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.EventLogTestHelper.convertEvent import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.{DRIVER_LOG_DFS_DIR, EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN} +import org.apache.spark.internal.config.{DRIVER_LOG_DFS_DIR, EVENT_LOG_COMPACTION_SCORE_THRESHOLD, EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN} import org.apache.spark.internal.config.History._ import org.apache.spark.internal.config.UI.{ADMIN_ACLS, ADMIN_ACLS_GROUPS, USER_GROUPS_MAPPING} import org.apache.spark.io._ @@ -55,6 +56,7 @@ import org.apache.spark.util.kvstore.InMemoryStore import org.apache.spark.util.logging.DriverLogger class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { + import EventLogTestHelper._ private var testDir: File = null @@ -167,8 +169,8 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { reader: EventLogFileReader, lastSeen: Long, enableSkipToEnd: Boolean, - compactible: Option[Boolean]): Unit = { - super.mergeApplicationListing(reader, lastSeen, enableSkipToEnd, compactible) + lastCompactionIndex: Option[Long]): Unit = { + super.mergeApplicationListing(reader, lastSeen, enableSkipToEnd, lastCompactionIndex) mergeApplicationListingCall += 1 } } @@ -1170,7 +1172,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { 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, None, Some(false), false) + Some("attemptId"), 100, None, None, false) var reader = EventLogFileReader(mockedFs, path) assert(reader.isDefined) assert(mockedProvider.shouldReloadLog(logInfo, reader.get)) @@ -1180,14 +1182,14 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { 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, None, Some(false), false) + Some("attemptId"), 100, None, 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, None, Some(false), false) + Some("attemptId"), 200, None, None, false) reader = EventLogFileReader(mockedFs, path) assert(reader.isDefined) assert(!mockedProvider.shouldReloadLog(logInfo, reader.get)) @@ -1299,7 +1301,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { assertSerDe(serializer, logInfoWithIndexAsNone) val logInfoWithIndex = LogInfo("dummy", 0, LogType.EventLogs, Some("appId"), - Some("attemptId"), 100, Some(3), Some(false), false) + Some("attemptId"), 100, Some(3), None, false) assertSerDe(serializer, logInfoWithIndex) } @@ -1316,11 +1318,11 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { val appInfo = new ApplicationAttemptInfo(None, new Date(1), new Date(1), new Date(1), 10, "spark", false, "dummy") val attemptInfoWithIndexAsNone = new AttemptInfoWrapper(appInfo, "dummyPath", 10, None, - None, None, None, None, None) + None, None, None, None) assertSerDe(serializer, attemptInfoWithIndexAsNone) val attemptInfoWithIndex = new AttemptInfoWrapper(appInfo, "dummyPath", 10, Some(1), - Some(true), None, None, None, None) + None, None, None, None) assertSerDe(serializer, attemptInfoWithIndex) } @@ -1339,177 +1341,65 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { test("compact event log files when replaying to rebuild app") { withTempDir { dir => val conf = createTestConf() + conf.set(HISTORY_LOG_DIR, dir.getAbsolutePath) conf.set(EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN, 1) + conf.set(EVENT_LOG_COMPACTION_SCORE_THRESHOLD, 0.0d) val hadoopConf = SparkHadoopUtil.newConfiguration(conf) val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + val writer = new RollingEventLogFilesWriter("app", None, dir.toURI, conf, hadoopConf) + writer.start() + // 1, 2 will be compacted into one file, 3 is the dummy file to ensure max files to retain - val logPath1 = EventLogTestHelper.writeEventLogFile(conf, hadoopConf, dir, 1, Seq( + writeEventsToRollingWriter(writer, Seq( SparkListenerApplicationStart("app", Some("app"), 0, "user", None), - SparkListenerJobStart(1, 0, Seq.empty))) - val logPath2 = EventLogTestHelper.writeEventLogFile(conf, hadoopConf, dir, 2, Seq( - SparkListenerUnpersistRDD(1), SparkListenerJobEnd(1, 1, JobSucceeded))) - val logPath3 = EventLogTestHelper.writeEventLogFile(conf, hadoopConf, dir, 3, Seq( + SparkListenerJobStart(1, 0, Seq.empty)), rollFile = true) + + writeEventsToRollingWriter(writer, Seq(SparkListenerUnpersistRDD(1), + SparkListenerJobEnd(1, 1, JobSucceeded)), rollFile = true) + + writeEventsToRollingWriter(writer, Seq( SparkListenerExecutorAdded(3, "exec1", new ExecutorInfo("host1", 1, Map.empty)), SparkListenerJobStart(2, 4, Seq.empty), - SparkListenerJobEnd(2, 5, JobSucceeded))) - val logPaths = Seq(logPath1, logPath2, logPath3) + SparkListenerJobEnd(2, 5, JobSucceeded)), rollFile = false) - val store = new InMemoryStore - val appStore = new AppStatusStore(store) - val reader = mock(classOf[EventLogFileReader]) - when(reader.listEventLogFiles).thenReturn(logPaths.map { p => fs.getFileStatus(new Path(p)) }) + writer.stop() val provider = new FsHistoryProvider(conf) - provider.rebuildAppStore(store, reader, 0, Some(true)) - - // files being compacted are deleted - assert(!fs.exists(new Path(logPath1))) - assert(!fs.exists(new Path(logPath2))) - - // new compact file should be available - assert(fs.exists(new Path(logPath2 + EventLogFileWriter.COMPACTED))) + updateAndCheck(provider) { _ => + val reader = EventLogFileReader(fs, new Path(writer.logPath)).get + val logFiles = reader.listEventLogFiles + // compacted file + retained file + assert(logFiles.size === 2) + assert(RollingEventLogFilesWriter.isEventLogFile(logFiles.head)) + assert(EventLogFileWriter.isCompacted(logFiles.head.getPath)) + assert(2 == RollingEventLogFilesWriter.getEventLogFileIndex(logFiles.head.getPath.getName)) + + assert(RollingEventLogFilesWriter.isEventLogFile(logFiles(1))) + assert(!EventLogFileWriter.isCompacted(logFiles(1).getPath)) + assert(3 == RollingEventLogFilesWriter.getEventLogFileIndex(logFiles(1).getPath.getName)) + + val store = new InMemoryStore + val appStore = new AppStatusStore(store) + + provider.rebuildAppStore(store, reader, 0L) + + // replayed store doesn't have any job, as events for job are removed while compacting + intercept[NoSuchElementException] { + appStore.job(1) + } - // retained file is not touched - assert(fs.exists(new Path(logPath3))) + // but other events should be available even they were in original files to compact + val appInfo = appStore.applicationInfo() + assert(appInfo.id === "app") + assert(appInfo.name === "app") - // replayed store doesn't have any job, as events for job are removed while compacting - intercept[NoSuchElementException] { - appStore.job(1) + // all events in retained file should be available, even they're related to finished jobs + val exec1 = appStore.executorSummary("exec1") + assert(exec1.hostPort === "host1") + val job2 = appStore.job(2) + assert(job2.status === JobExecutionStatus.SUCCEEDED) } - - // but other events should be available even they were in original files to compact - val appInfo = appStore.applicationInfo() - assert(appInfo.id === "app") - assert(appInfo.name === "app") - - // all events in retained file should be available, even they're related to finished jobs - val exec1 = appStore.executorSummary("exec1") - assert(exec1.hostPort === "host1") - val job2 = appStore.job(2) - assert(job2.status === JobExecutionStatus.SUCCEEDED) - } - } - - class EligibilityCallTrackingFsHistoryProvider extends FsHistoryProvider(createTestConf()) { - var checkEligibilityForCompactionCall = 0 - override protected def checkEligibilityForCompaction( - info: LogInfo, - reader: EventLogFileReader): Option[Boolean] = { - val ret = super.checkEligibilityForCompaction(info, reader) - checkEligibilityForCompactionCall += 1 - ret - } - } - - private def createLogInfo(reader: EventLogFileReader, appId: String): LogInfo = { - LogInfo(reader.rootPath.toString, 0L, LogType.EventLogs, - Some(appId), None, reader.fileSizeForLastIndex, reader.lastIndex, - None, reader.completed) - } - - private def assertCompactibleInLogInfo( - provider: FsHistoryProvider, - reader: EventLogFileReader, - expected: Option[Boolean]): Unit = { - val logInfo = provider.listing.read(classOf[LogInfo], reader.rootPath.toString) - assert(expected === logInfo.compactible) - } - - test("update eligibility for compaction - rolling event log") { - def mockReaderForRollingEventLog( - fs: FileSystem, - rootPath: Path, - logPaths: Seq[Path]): EventLogFileReader = { - require(logPaths.nonEmpty) - val reader = mock(classOf[EventLogFileReader]) - val statuses = logPaths.map { log => fs.getFileStatus(log) } - when(reader.listEventLogFiles).thenReturn(statuses) - when(reader.lastIndex).thenReturn(Some(1L)) - when(reader.rootPath).thenReturn(rootPath) - when(reader.fileSizeForLastIndex).thenReturn(statuses.last.getLen) - when(reader.completed).thenReturn(false) - reader - } - - withTempDir { dir => - val provider = new EligibilityCallTrackingFsHistoryProvider - - val conf = createTestConf() - val hadoopConf = SparkHadoopUtil.newConfiguration(conf) - val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - - val logPath1 = EventLogTestHelper.writeEventLogFile(conf, hadoopConf, dir, 1, Seq( - SparkListenerApplicationStart("app", Some("app"), 0, "user", None))) - - val reader1 = mockReaderForRollingEventLog(fs, new Path(dir.getCanonicalPath), - Seq(new Path(logPath1))) - - provider.listing.write(createLogInfo(reader1, "app")) - provider.getOrUpdateCompactible(reader1) - - assert(1 === provider.checkEligibilityForCompactionCall) - // All events in log file are expected to be accepted - so 'compactible' should be true, - // but we don't decide if there's only one file given the file might be updated later. - assertCompactibleInLogInfo(provider, reader1, None) - - // 10 of 11 events will be rejected which meets the condition of compaction - val logPath2 = EventLogTestHelper.writeEventLogFile(conf, hadoopConf, dir, 2, - (1 to 5).flatMap { idx => - Seq(SparkListenerJobStart(idx, 1L, Seq.empty, null), - SparkListenerJobEnd(idx, 1L, JobSucceeded)) - } ++ Seq(SparkListenerUnpersistRDD(1))) - - val reader2 = mockReaderForRollingEventLog(fs, new Path(dir.getCanonicalPath), - Seq(new Path(logPath1), new Path(logPath2))) - - provider.listing.write(createLogInfo(reader2, "app")) - provider.checkEligibilityForCompactionCall = 0 - provider.getOrUpdateCompactible(reader2) - - assert(1 === provider.checkEligibilityForCompactionCall) - // All events in log file are expected to be accepted - so 'compactible' should be false. - assertCompactibleInLogInfo(provider, reader2, Some(false)) - - val logPath3 = EventLogTestHelper.writeEventLogFile(conf, hadoopConf, dir, 3, Seq( - SparkListenerApplicationEnd(4L))) - - val reader3 = mockReaderForRollingEventLog(fs, new Path(dir.getCanonicalPath), - Seq(new Path(logPath2), new Path(logPath3))) - - provider.listing.write(createLogInfo(reader3, "app")) - provider.checkEligibilityForCompactionCall = 0 - provider.getOrUpdateCompactible(reader3) - - assert(1 === provider.checkEligibilityForCompactionCall) - // Most of events in log file are expected to be rejected - so 'compactible' should - // be true. - assertCompactibleInLogInfo(provider, reader3, Some(true)) - } - } - - test("update eligibility for compaction - single event log") { - withTempDir { dir => - val provider = new EligibilityCallTrackingFsHistoryProvider - - val conf = createTestConf() - val hadoopConf = SparkHadoopUtil.newConfiguration(conf) - val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - - val singleLogFile = newLogFile("new1", None, inProgress = false) - writeFile(singleLogFile, None, - SparkListenerApplicationStart(singleLogFile.getName(), Some("new-app-complete"), 1L, "test", - None), - SparkListenerApplicationEnd(5L) - ) - - val readerForSingleLog = EventLogFileReader(fs, new Path(singleLogFile.getCanonicalPath)).get - provider.listing.write(createLogInfo(readerForSingleLog, "new1")) - - provider.getOrUpdateCompactible(readerForSingleLog) - - assert(1 === provider.checkEligibilityForCompactionCall) - assertCompactibleInLogInfo(provider, readerForSingleLog, Some(false)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala index 82213f34b84d5..460420d67a99e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala @@ -120,7 +120,7 @@ private[spark] class SQLLiveEntitiesEventFilter( _jobToStages: Map[Int, Seq[Int]], _stageToTasks: Map[Int, Set[Long]], _stageToRDDs: Map[Int, Seq[Int]]) - extends JobEventFilter(_jobToStages, _stageToTasks, _stageToRDDs) with Logging { + extends JobEventFilter(None, _jobToStages, _stageToTasks, _stageToRDDs) with Logging { logDebug(s"live executions : ${liveExecutionToJobs.keySet}") logDebug(s"jobs in live executions : ${liveExecutionToJobs.values.flatten}") From 761833ba4d93f833c4a25f12bf68561528fe7a88 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 23 Dec 2019 16:11:50 +0900 Subject: [PATCH 27/28] Address incremental load on constructing EventFilterBuilder --- .../history/EventLogFileCompactor.scala | 132 ++++++++--- .../deploy/history/FsHistoryProvider.scala | 60 +++-- .../history/EventLogFileCompactorSuite.scala | 220 ++++++++++++++---- .../deploy/history/EventLogTestHelper.scala | 5 +- .../history/FsHistoryProviderSuite.scala | 71 ++++-- 5 files changed, 368 insertions(+), 120 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index 581d3335d3a03..d3e05e2748aca 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -21,6 +21,7 @@ import java.io.IOException import java.net.URI import java.util.ServiceLoader +import scala.annotation.tailrec import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration @@ -28,6 +29,7 @@ import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.spark.SparkConf import org.apache.spark.deploy.history.EventFilter.FilterStatistic +import org.apache.spark.deploy.history.EventFilterBuildersLoader.LowerIndexLoadRequested import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{EVENT_LOG_COMPACTION_SCORE_THRESHOLD, EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN} import org.apache.spark.scheduler.ReplayListenerBus @@ -46,10 +48,6 @@ import org.apache.spark.util.Utils * represents approximate rate of filtered-out events. Score is being calculated via applying * heuristic; task events tend to take most size in event log. * - * This class assumes caller will provide the sorted list of files which are sorted by the index of - * event log file, with "at most" one compact file placed first if it exists. Caller should keep in - * mind that this class doesn't care about the semantic of ordering. - * * When compacting the files, the range of compaction for given file list is determined as: * (first ~ the file where there're `maxFilesToRetain` files on the right side) * @@ -59,22 +57,43 @@ class EventLogFileCompactor( sparkConf: SparkConf, hadoopConf: Configuration, fs: FileSystem) extends Logging { + import EventFilterBuildersLoader._ + private val maxFilesToRetain: Int = sparkConf.get(EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN) private val compactionThresholdScore: Double = sparkConf.get(EVENT_LOG_COMPACTION_SCORE_THRESHOLD) - def compact(eventLogFiles: Seq[FileStatus]): (CompactionResult.Value, Option[Long]) = { - assertPrecondition(eventLogFiles) + private var filterBuildersLoader = new EventFilterBuildersLoader(fs) + private var loadedLogPath: Path = _ + + def compact(reader: EventLogFileReader): (CompactionResult.Value, Option[Long]) = { + doCompact(reader) + } + + @tailrec + private def doCompact(reader: EventLogFileReader): (CompactionResult.Value, Option[Long]) = { + if (loadedLogPath == null) { + loadedLogPath = reader.rootPath + } else { + require(loadedLogPath == null || reader.rootPath == loadedLogPath, + "An instance of compactor should deal with same path of event log.") + } + + if (reader.lastIndex.isEmpty) { + return (CompactionResult.NOT_ENOUGH_FILES, None) + } + val eventLogFiles = reader.listEventLogFiles if (eventLogFiles.length < maxFilesToRetain) { return (CompactionResult.NOT_ENOUGH_FILES, None) } val filesToCompact = findFilesToCompact(eventLogFiles) if (filesToCompact.isEmpty) { - (CompactionResult.NOT_ENOUGH_FILES, None) - } else { - val builders = initializeBuilders(fs, filesToCompact.map(_.getPath)) + return (CompactionResult.NOT_ENOUGH_FILES, None) + } + try { + val builders = filterBuildersLoader.loadNewFiles(filesToCompact) val filters = builders.map(_.createFilter()) val minScore = filters.flatMap(_.statistic()).map(calculateScore).min @@ -87,37 +106,14 @@ class EventLogFileCompactor( (CompactionResult.SUCCESS, Some(RollingEventLogFilesWriter.getEventLogFileIndex( filesToCompact.last.getPath.getName))) } + } catch { + case _: LowerIndexLoadRequested => + // reset loader and load again + filterBuildersLoader = new EventFilterBuildersLoader(fs) + doCompact(reader) } } - private def assertPrecondition(eventLogFiles: Seq[FileStatus]): Unit = { - val idxCompactedFiles = eventLogFiles.zipWithIndex.filter { case (file, _) => - EventLogFileWriter.isCompacted(file.getPath) - } - require(idxCompactedFiles.size < 2 && idxCompactedFiles.headOption.forall(_._2 == 0), - "The number of compact files should be at most 1, and should be placed first if exists.") - } - - /** - * Loads all available EventFilterBuilders in classloader via ServiceLoader, and initializes - * them via replaying events in given files. - */ - private def initializeBuilders(fs: FileSystem, files: Seq[Path]): Seq[EventFilterBuilder] = { - val bus = new ReplayListenerBus() - - val builders = ServiceLoader.load(classOf[EventFilterBuilder], - Utils.getContextOrSparkClassLoader).asScala.toSeq - builders.foreach(bus.addListener) - - files.foreach { log => - Utils.tryWithResource(EventLogFileReader.openEventLog(log, fs)) { in => - bus.replay(in, log.getName) - } - } - - builders - } - private def calculateScore(stats: FilterStatistic): Double = { // For now it's simply measuring how many task events will be filtered out (rejected) // but it can be sophisticated later once we get more heuristic information and found @@ -162,6 +158,68 @@ object CompactionResult extends Enumeration { val SUCCESS, NOT_ENOUGH_FILES, LOW_SCORE_FOR_COMPACTION = Value } +class EventFilterBuildersLoader(fs: FileSystem) { + // the implementation of this bus is expected to be stateless + private val bus = new ReplayListenerBus() + + /** Loads all available EventFilterBuilders in classloader via ServiceLoader */ + private val filterBuilders: Seq[EventFilterBuilder] = ServiceLoader.load( + classOf[EventFilterBuilder], Utils.getContextOrSparkClassLoader).asScala.toSeq + + filterBuilders.foreach(bus.addListener) + + private var latestIndexLoaded: Long = -1L + + /** only exposed for testing; simple metric to help testing */ + private[history] var numFilesToLoad: Long = 0L + + /** + * Initializes EventFilterBuilders via replaying events in given files. Loading files are done + * incrementally, via dropping indices which are already loaded and replaying remaining files. + * For example, If the last index of requested files is same as the last index being loaded, + * this will not replay any files. + * + * If the last index of requested files is smaller than the last index being loaded, it will + * throw [[LowerIndexLoadRequested]], which caller can decide whether ignoring it or + * invalidating loader and retrying. + */ + def loadNewFiles(eventLogFiles: Seq[FileStatus]): Seq[EventFilterBuilder] = { + require(eventLogFiles.nonEmpty) + + val idxToStatuses = eventLogFiles.map { status => + val idx = RollingEventLogFilesWriter.getEventLogFileIndex(status.getPath.getName) + idx -> status + } + + val newLatestIdx = idxToStatuses.last._1 + if (newLatestIdx < latestIndexLoaded) { + throw new LowerIndexLoadRequested("Loader already loads higher index of event log than" + + " requested.") + } + + val filesToLoad = idxToStatuses + .filter { case (idx, _) => idx > latestIndexLoaded } + .map { case (_, status) => status.getPath } + + if (filesToLoad.nonEmpty) { + filesToLoad.foreach { log => + Utils.tryWithResource(EventLogFileReader.openEventLog(log, fs)) { in => + bus.replay(in, log.getName) + } + numFilesToLoad += 1 + } + + latestIndexLoaded = newLatestIdx + } + + filterBuilders + } +} + +object EventFilterBuildersLoader { + class LowerIndexLoadRequested(_msg: String) extends Exception(_msg) +} + /** * This class rewrites the event log files into one compact file: the compact file will only * contain the events which pass the filters. Events will be dropped only when all filters 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 723ffb793d6d4..31fc8beb68115 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 @@ -159,7 +159,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) new HistoryServerDiskManager(conf, path, listing, clock) } - private val fileCompactor = new EventLogFileCompactor(conf, hadoopConf, fs) + // Visible for testing. + private[history] val logToCompactor = new mutable.HashMap[String, EventLogFileCompactor] // Used to store the paths, which are being processed. This enable the replay log tasks execute // asynchronously and make sure that checkForLogs would not process a path repeatedly. @@ -526,20 +527,26 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) reader.fileSizeForLastIndex > 0 } } - .sortWith { case (entry1, entry2) => - entry1.modificationTime > entry2.modificationTime + .sortWith { case (reader1, reader2) => + reader1.modificationTime > reader2.modificationTime } if (updated.nonEmpty) { logDebug(s"New/updated attempts found: ${updated.size} ${updated.map(_.rootPath)}") } - updated.foreach { entry => - processing(entry.rootPath) + updated.foreach { reader => + processing(reader.rootPath) try { val task: Runnable = () => { - val updatedLastCompactionIndex = compact(entry) - mergeApplicationListing(entry, newLastScanTime, true, updatedLastCompactionIndex) + val (shouldRenewReader, updatedLastCompactionIndex) = compact(reader) + // we should renew reader if the list of event log files are changed in `compact` + val newReader = if (shouldRenewReader) { + EventLogFileReader(fs, reader.rootPath).get + } else { + reader + } + mergeApplicationListing(newReader, newLastScanTime, true, updatedLastCompactionIndex) } replayExecutor.submit(task) } catch { @@ -566,6 +573,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) log.appId.foreach { appId => cleanAppData(appId, log.attemptId, log.logPath) listing.delete(classOf[LogInfo], log.logPath) + cleanupCompactor(log.logPath) } } @@ -576,26 +584,33 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } /** exposed for testing */ - private[history] def compact(reader: EventLogFileReader): Option[Long] = { + private[history] def compact(reader: EventLogFileReader): (Boolean, Option[Long]) = { reader.lastIndex match { case Some(lastIndex) => try { - val info = listing.read(classOf[LogInfo], reader.rootPath.toString) + val rootPath = reader.rootPath.toString + val info = listing.read(classOf[LogInfo], rootPath) if (info.lastCompactionIndex.isEmpty || info.lastCompactionIndex.get < lastIndex) { // haven't tried compaction for this index, do compaction - val (_, lastCompactionIndex) = fileCompactor.compact(reader.listEventLogFiles) - listing.write(info.copy(lastCompactionIndex = lastCompactionIndex)) - Some(lastIndex) + val compactor = logToCompactor.getOrElseUpdate(rootPath, + new EventLogFileCompactor(conf, hadoopConf, fs)) + val (compactionResult, lastCompactionIndex) = compactor.compact(reader) + if (compactionResult == CompactionResult.SUCCESS) { + listing.write(info.copy(lastCompactionIndex = lastCompactionIndex)) + (true, Some(lastIndex)) + } else { + (false, Some(lastIndex)) + } } else { - info.lastCompactionIndex + (false, info.lastCompactionIndex) } } catch { case _: NoSuchElementException => // this should exist, but ignoring doesn't hurt much - None + (false, None) } - case None => None // This is not applied to single event log file. + case None => (false, None) // This is not applied to single event log file. } } @@ -702,12 +717,14 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) case e: InterruptedException => throw e case e: AccessControlException => + val rootPath = reader.rootPath // We don't have read permissions on the log file - logWarning(s"Unable to read log ${reader.rootPath}", e) - blacklist(reader.rootPath) + logWarning(s"Unable to read log ${rootPath}", e) + blacklist(rootPath) // SPARK-28157 We should remove this blacklisted entry from the KVStore // to handle permission-only changes with the same file sizes later. - listing.delete(classOf[LogInfo], reader.rootPath.toString) + listing.delete(classOf[LogInfo], rootPath.toString) + cleanupCompactor(rootPath.toString) case e: Exception => logError("Exception while merging application listings", e) } finally { @@ -864,6 +881,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) logInfo(s"Deleting invalid / corrupt event log ${log.logPath}") deleteLog(fs, new Path(log.logPath)) listing.delete(classOf[LogInfo], log.logPath) + cleanupCompactor(log.logPath) } log.appId.foreach { appId => @@ -911,6 +929,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) logInfo(s"Deleting invalid / corrupt event log ${log.logPath}") deleteLog(fs, new Path(log.logPath)) listing.delete(classOf[LogInfo], log.logPath) + cleanupCompactor(log.logPath) } } @@ -953,6 +972,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) logInfo(s"Deleting expired event log for ${attempt.logPath}") val logPath = new Path(logDir, attempt.logPath) listing.delete(classOf[LogInfo], logPath.toString()) + cleanupCompactor(logPath.toString) cleanAppData(app.id, attempt.info.attemptId, logPath.toString()) if (deleteLog(fs, logPath)) { countDeleted += 1 @@ -1240,6 +1260,10 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } deleted } + + private def cleanupCompactor(logPath: String): Unit = { + logToCompactor -= logPath + } } private[history] object FsHistoryProvider { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala index 882a5f183ee5b..0dbfd29239fd7 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala @@ -17,12 +17,11 @@ package org.apache.spark.deploy.history -import java.io.File - import scala.io.{Codec, Source} import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.json4s.jackson.JsonMethods.parse +import org.scalatest.PrivateMethodTester import org.apache.spark.{SparkConf, SparkFunSuite, Success} import org.apache.spark.deploy.SparkHadoopUtil @@ -34,30 +33,21 @@ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.status.ListenerEventsTestHelper import org.apache.spark.util.{JsonProtocol, Utils} -class EventLogFileCompactorSuite extends SparkFunSuite { +class EventLogFileCompactorSuite extends SparkFunSuite with PrivateMethodTester { import ListenerEventsTestHelper._ private val sparkConf = testSparkConf() private val hadoopConf = SparkHadoopUtil.newConfiguration(sparkConf) - test("No event log files") { - withTempDir { dir => - val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) - - assertNoCompaction(fs, Seq.empty, compactor.compact(Seq.empty), - CompactionResult.NOT_ENOUGH_FILES) - } - } - test("No compact file, less origin files available than max files to retain") { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + val logPath = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, (1 to 2).map(_ => testEvent): _*) + val reader = EventLogFileReader(fs, new Path(logPath)).get val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) - assertNoCompaction(fs, fileStatuses, compactor.compact(fileStatuses), + assertNoCompaction(fs, reader.listEventLogFiles, compactor.compact(reader), CompactionResult.NOT_ENOUGH_FILES) } } @@ -66,10 +56,12 @@ class EventLogFileCompactorSuite extends SparkFunSuite { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + val logPath = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, (1 to 5).map(_ => testEvent): _*) + + val reader = EventLogFileReader(fs, new Path(logPath)).get val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) - assertCompaction(fs, fileStatuses, compactor.compact(fileStatuses), + assertCompaction(fs, reader.listEventLogFiles, compactor.compact(reader), expectedNumOfFilesCompacted = 2) } } @@ -78,17 +70,14 @@ class EventLogFileCompactorSuite extends SparkFunSuite { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + val logPath = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, (1 to 2).map(_ => testEvent): _*) - val fileToCompact = fileStatuses.head.getPath - val compactedPath = new Path(fileToCompact.getParent, - fileToCompact.getName + EventLogFileWriter.COMPACTED) - assert(fs.rename(fileToCompact, compactedPath)) + fakeCompactFirstEventLogFile(fs, logPath) - val newFileStatuses = Seq(fs.getFileStatus(compactedPath)) ++ fileStatuses.drop(1) + val newReader = EventLogFileReader(fs, new Path(logPath)).get val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) - assertNoCompaction(fs, newFileStatuses, compactor.compact(newFileStatuses), + assertNoCompaction(fs, newReader.listEventLogFiles, compactor.compact(newReader), CompactionResult.NOT_ENOUGH_FILES) } } @@ -97,17 +86,14 @@ class EventLogFileCompactorSuite extends SparkFunSuite { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + val logPath = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, (1 to 4).map(_ => testEvent): _*) - val fileToCompact = fileStatuses.head.getPath - val compactedPath = new Path(fileToCompact.getParent, - fileToCompact.getName + EventLogFileWriter.COMPACTED) - assert(fs.rename(fileToCompact, compactedPath)) + fakeCompactFirstEventLogFile(fs, logPath) - val newFileStatuses = Seq(fs.getFileStatus(compactedPath)) ++ fileStatuses.drop(1) + val newReader = EventLogFileReader(fs, new Path(logPath)).get val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) - assertNoCompaction(fs, newFileStatuses, compactor.compact(newFileStatuses), + assertNoCompaction(fs, newReader.listEventLogFiles, compactor.compact(newReader), CompactionResult.NOT_ENOUGH_FILES) } } @@ -116,27 +102,33 @@ class EventLogFileCompactorSuite extends SparkFunSuite { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + val logPath = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, (1 to 10).map(_ => testEvent): _*) - val fileToCompact = fileStatuses.head.getPath - val compactedPath = new Path(fileToCompact.getParent, - fileToCompact.getName + EventLogFileWriter.COMPACTED) - assert(fs.rename(fileToCompact, compactedPath)) + fakeCompactFirstEventLogFile(fs, logPath) - val newFileStatuses = Seq(fs.getFileStatus(compactedPath)) ++ fileStatuses.drop(1) + val newReader = EventLogFileReader(fs, new Path(logPath)).get val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) - assertCompaction(fs, newFileStatuses, compactor.compact(newFileStatuses), + assertCompaction(fs, newReader.listEventLogFiles, compactor.compact(newReader), expectedNumOfFilesCompacted = 7) } } + private def fakeCompactFirstEventLogFile(fs: FileSystem, logPath: String): Unit = { + val reader = EventLogFileReader(fs, new Path(logPath)).get + val fileStatuses = reader.listEventLogFiles + val fileToCompact = fileStatuses.head.getPath + val compactedPath = new Path(fileToCompact.getParent, + fileToCompact.getName + EventLogFileWriter.COMPACTED) + assert(fs.rename(fileToCompact, compactedPath)) + } + test("events for finished job are dropped in new compact file") { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) // 1, 2 will be compacted into one file, 3~5 are dummies to ensure max files to retain - val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + val logPath = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, Seq( SparkListenerExecutorAdded(0, "exec1", new ExecutorInfo("host1", 1, Map.empty)), SparkListenerJobStart(1, 0, Seq.empty)), @@ -147,11 +139,12 @@ class EventLogFileCompactorSuite extends SparkFunSuite { testEvent, testEvent) + val reader = EventLogFileReader(fs, new Path(logPath)).get val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) - assertCompaction(fs, fileStatuses, compactor.compact(fileStatuses), + assertCompaction(fs, reader.listEventLogFiles, compactor.compact(reader), expectedNumOfFilesCompacted = 2) - val expectCompactFileBasePath = fileStatuses.take(2).last.getPath + val expectCompactFileBasePath = reader.listEventLogFiles.take(2).last.getPath val compactFilePath = getCompactFilePath(expectCompactFileBasePath) Utils.tryWithResource(EventLogFileReader.openEventLog(compactFilePath, fs)) { is => val lines = Source.fromInputStream(is)(Codec.UTF8).getLines().toList @@ -173,7 +166,7 @@ class EventLogFileCompactorSuite extends SparkFunSuite { // only one of two tasks is finished, which would score 0.5d val tasks = createTasks(2, Array("exec1"), 0L).map(createTaskStartEvent(_, 1, 0)) - val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, newConf, hadoopConf, + val logPath = writeEventsToRollingWriter(fs, "app", dir, newConf, hadoopConf, tasks, Seq(SparkListenerTaskEnd(1, 0, "taskType", Success, tasks.head.taskInfo, new ExecutorMetrics, null)), @@ -181,12 +174,153 @@ class EventLogFileCompactorSuite extends SparkFunSuite { testEvent, testEvent) + val reader = EventLogFileReader(fs, new Path(logPath)).get val compactor = new EventLogFileCompactor(newConf, hadoopConf, fs) - assertNoCompaction(fs, fileStatuses, compactor.compact(fileStatuses), + assertNoCompaction(fs, reader.listEventLogFiles, compactor.compact(reader), CompactionResult.LOW_SCORE_FOR_COMPACTION) } } + test("incremental load of event filter builder") { + val loaderMethod = PrivateMethod[EventFilterBuildersLoader](Symbol("filterBuildersLoader")) + + withTempDir { dir => + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + val writer = new RollingEventLogFilesWriter("app", None, dir.toURI, sparkConf, hadoopConf) + writer.start() + + // write 1-5, and 6 + (1 to 5).map(_ => testEvent).foreach { events => + writeEventsToRollingWriter(writer, events, rollFile = true) + } + writeEventsToRollingWriter(writer, testEvent, rollFile = false) + + // don't stop writer as we will add more files later + val logPath = writer.logPath + + val reader = EventLogFileReader(fs, new Path(logPath)).get + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + assertCompaction(fs, reader.listEventLogFiles, compactor.compact(reader), + expectedNumOfFilesCompacted = 3) + + val loader = compactor.invokePrivate(loaderMethod()) + assert(loader.numFilesToLoad === 3) + + // event log files: 3.compact, 4, 5, 6 + // add two more log files (7, 8) to compact 3, 4, 5 as 5.compact + + // just roll out without writing to finalize 6, and write 7 and 8 + writeEventsToRollingWriter(writer, Seq.empty, rollFile = true) + writeEventsToRollingWriter(writer, testEvent, rollFile = true) + writeEventsToRollingWriter(writer, testEvent, rollFile = false) + + writer.stop() + + val reader2 = EventLogFileReader(fs, new Path(logPath)).get + assertCompaction(fs, reader2.listEventLogFiles, compactor.compact(reader2), + expectedNumOfFilesCompacted = 3) + + val loader2 = compactor.invokePrivate(loaderMethod()) + // 3 + 2 (no need to re-read compacted file as the state would be same after compaction) + assert(loader2.numFilesToLoad === 5) + } + } + + test("request compaction with lower index than already requested") { + val loaderMethod = PrivateMethod[EventFilterBuildersLoader](Symbol("filterBuildersLoader")) + + withTempDir { dir => + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + val logPath = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + (1 to 6).map(_ => testEvent): _*) + + val reader = EventLogFileReader(fs, new Path(logPath)).get + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + assertCompaction(fs, reader.listEventLogFiles, compactor.compact(reader), + expectedNumOfFilesCompacted = 3) + + val loader = compactor.invokePrivate(loaderMethod()) + assert(loader.numFilesToLoad === 3) + + // remove directory and reconstruct files with one less file which makes compactor to target + // less index to compact; note that this shouldn't happen in happy situation, but assuming + // the bad cases here. + assert(fs.delete(new Path(logPath), true)) + val logPath2 = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + (1 to 5).map(_ => testEvent): _*) + assert(logPath === logPath2) + + val reader2 = EventLogFileReader(fs, new Path(logPath2)).get + // compactor will reload the log files and compact the necessary files + assertCompaction(fs, reader2.listEventLogFiles, compactor.compact(reader2), + expectedNumOfFilesCompacted = 2) + + val loader2 = compactor.invokePrivate(loaderMethod()) + assert(loader ne loader2) + assert(loader2.numFilesToLoad === 2) + } + } + + test("request compaction multiple times with same event log files") { + val loaderMethod = PrivateMethod[EventFilterBuildersLoader](Symbol("filterBuildersLoader")) + + withTempDir { dir => + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + val logPath = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + (1 to 6).map(_ => testEvent): _*) + + val reader = EventLogFileReader(fs, new Path(logPath)).get + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + assertCompaction(fs, reader.listEventLogFiles, compactor.compact(reader), + expectedNumOfFilesCompacted = 3) + + val loader = compactor.invokePrivate(loaderMethod()) + assert(loader.numFilesToLoad === 3) + + // remove directory and reconstruct files which effectively reverts the previous compaction; + // note that this shouldn't happen in happy situation, but assuming the bad cases here. + assert(fs.delete(new Path(logPath), true)) + val logPath2 = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + (1 to 6).map(_ => testEvent): _*) + assert(logPath === logPath2) + + val reader2 = EventLogFileReader(fs, new Path(logPath2)).get + // compactor will reload the log files and compact the necessary files + assertCompaction(fs, reader2.listEventLogFiles, compactor.compact(reader2), + expectedNumOfFilesCompacted = 3) + + val loader2 = compactor.invokePrivate(loaderMethod()) + assert(loader eq loader2) + // no new file should be read + assert(loader2.numFilesToLoad === 3) + } + } + + test("request compaction with different log paths") { + withTempDir { dir => + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + val logPath = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, + (1 to 6).map(_ => testEvent): _*) + + val reader = EventLogFileReader(fs, new Path(logPath)).get + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + assertCompaction(fs, reader.listEventLogFiles, compactor.compact(reader), + expectedNumOfFilesCompacted = 3) + + val logPath2 = writeEventsToRollingWriter(fs, "app2", dir, sparkConf, hadoopConf, + (1 to 6).map(_ => testEvent): _*) + + val reader2 = EventLogFileReader(fs, new Path(logPath2)).get + intercept[IllegalArgumentException] { + compactor.compact(reader2) + } + } + } + private def assertCompaction( fs: FileSystem, originalFiles: Seq[FileStatus], 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 66e3fb788939e..f5de18babca8d 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 @@ -83,7 +83,7 @@ object EventLogTestHelper { dir: File, sparkConf: SparkConf, hadoopConf: Configuration, - eventsFiles: Seq[SparkListenerEvent]*): Seq[FileStatus] = { + eventsFiles: Seq[SparkListenerEvent]*): String = { val writer = new RollingEventLogFilesWriter(appId, None, dir.toURI, sparkConf, hadoopConf) writer.start() @@ -95,7 +95,8 @@ object EventLogTestHelper { } writer.stop() - EventLogFileReader(fs, new Path(writer.logPath)).get.listEventLogFiles + + writer.logPath } def writeEventsToRollingWriter( 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 fd60c13adc883..af6daa9a50c98 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 @@ -28,6 +28,7 @@ import scala.concurrent.duration._ import com.google.common.io.{ByteStreams, Files} import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, FSDataInputStream, Path} import org.apache.hadoop.hdfs.{DFSInputStream, DistributedFileSystem} import org.apache.hadoop.security.AccessControlException @@ -39,7 +40,6 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.{JobExecutionStatus, SecurityManager, SPARK_VERSION, SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.history.EventLogTestHelper.convertEvent import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{DRIVER_LOG_DFS_DIR, EVENT_LOG_COMPACTION_SCORE_THRESHOLD, EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN} import org.apache.spark.internal.config.History._ @@ -1376,27 +1376,13 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { val hadoopConf = SparkHadoopUtil.newConfiguration(conf) val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - val writer = new RollingEventLogFilesWriter("app", None, dir.toURI, conf, hadoopConf) - writer.start() - - // 1, 2 will be compacted into one file, 3 is the dummy file to ensure max files to retain - writeEventsToRollingWriter(writer, Seq( - SparkListenerApplicationStart("app", Some("app"), 0, "user", None), - SparkListenerJobStart(1, 0, Seq.empty)), rollFile = true) - - writeEventsToRollingWriter(writer, Seq(SparkListenerUnpersistRDD(1), - SparkListenerJobEnd(1, 1, JobSucceeded)), rollFile = true) - - writeEventsToRollingWriter(writer, Seq( - SparkListenerExecutorAdded(3, "exec1", new ExecutorInfo("host1", 1, Map.empty)), - SparkListenerJobStart(2, 4, Seq.empty), - SparkListenerJobEnd(2, 5, JobSucceeded)), rollFile = false) - - writer.stop() + val logPath = constructEventLogsForCompactionTest(fs, "app", dir, conf, hadoopConf) val provider = new FsHistoryProvider(conf) - updateAndCheck(provider) { _ => - val reader = EventLogFileReader(fs, new Path(writer.logPath)).get + updateAndCheck(provider) { listing => + assert(listing.map(_.id).toSet === Set("app")) + + val reader = EventLogFileReader(fs, new Path(logPath)).get val logFiles = reader.listEventLogFiles // compacted file + retained file assert(logFiles.size === 2) @@ -1432,6 +1418,51 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { } } + test("The compactor instance will be reserved per app log") { + withTempDir { dir => + val conf = createTestConf() + conf.set(HISTORY_LOG_DIR, dir.getAbsolutePath) + conf.set(EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN, 1) + conf.set(EVENT_LOG_COMPACTION_SCORE_THRESHOLD, 0.0d) + val hadoopConf = SparkHadoopUtil.newConfiguration(conf) + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + val app1LogPath = constructEventLogsForCompactionTest(fs, "app1", dir, conf, hadoopConf) + val app2LogPath = constructEventLogsForCompactionTest(fs, "app2", dir, conf, hadoopConf) + val app3LogPath = constructEventLogsForCompactionTest(fs, "app3", dir, conf, hadoopConf) + + val provider = new FsHistoryProvider(conf) + updateAndCheck(provider) { listing => + assert(listing.map(_.id).toSet === Set("app1", "app2", "app3")) + + val compactorForApp1 = provider.logToCompactor(app1LogPath) + val compactorForApp2 = provider.logToCompactor(app2LogPath) + val compactorForApp3 = provider.logToCompactor(app3LogPath) + assert(compactorForApp1 ne compactorForApp2) + assert(compactorForApp2 ne compactorForApp3) + assert(compactorForApp1 ne compactorForApp3) + } + } + } + + private def constructEventLogsForCompactionTest( + fs: FileSystem, + appId: String, + dir: File, + conf: SparkConf, + hadoopConf: Configuration): String = { + writeEventsToRollingWriter(fs, appId, dir, conf, hadoopConf, + // 1, 2 will be compacted into one file, 3 is the dummy file to ensure max files to retain + Seq( + SparkListenerApplicationStart(appId, Some(appId), 0, "user", None), + SparkListenerJobStart(1, 0, Seq.empty)), + Seq(SparkListenerUnpersistRDD(1), SparkListenerJobEnd(1, 1, JobSucceeded)), + Seq( + SparkListenerExecutorAdded(3, "exec1", new ExecutorInfo("host1", 1, Map.empty)), + SparkListenerJobStart(2, 4, Seq.empty), + SparkListenerJobEnd(2, 5, JobSucceeded))) + } + /** * Asks the provider to check for logs and calls a function to perform checks on the updated * app list. Example: From e1a6e42b73f8d58dbc0a04882f2288d2fae0dad8 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 26 Dec 2019 11:34:45 +0900 Subject: [PATCH 28/28] Reset EventFilterBuildersLoader for any exceptions --- .../history/EventLogFileCompactor.scala | 44 ++++++++++--------- .../deploy/history/EventLogFileReaders.scala | 20 ++++++--- 2 files changed, 38 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index d3e05e2748aca..ce50e7e132524 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -21,8 +21,8 @@ import java.io.IOException import java.net.URI import java.util.ServiceLoader -import scala.annotation.tailrec import scala.collection.JavaConverters._ +import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} @@ -66,11 +66,6 @@ class EventLogFileCompactor( private var loadedLogPath: Path = _ def compact(reader: EventLogFileReader): (CompactionResult.Value, Option[Long]) = { - doCompact(reader) - } - - @tailrec - private def doCompact(reader: EventLogFileReader): (CompactionResult.Value, Option[Long]) = { if (loadedLogPath == null) { loadedLogPath = reader.rootPath } else { @@ -92,25 +87,34 @@ class EventLogFileCompactor( return (CompactionResult.NOT_ENOUGH_FILES, None) } - try { - val builders = filterBuildersLoader.loadNewFiles(filesToCompact) - val filters = builders.map(_.createFilter()) - val minScore = filters.flatMap(_.statistic()).map(calculateScore).min + val builders = loadFilesToFilterBuilder(filesToCompact) + val filters = builders.map(_.createFilter()) + val minScore = filters.flatMap(_.statistic()).map(calculateScore).min - if (minScore < compactionThresholdScore) { - (CompactionResult.LOW_SCORE_FOR_COMPACTION, None) - } else { - val rewriter = new FilteredEventLogFileRewriter(sparkConf, hadoopConf, fs, filters) - rewriter.rewrite(filesToCompact) - cleanupCompactedFiles(filesToCompact) - (CompactionResult.SUCCESS, Some(RollingEventLogFilesWriter.getEventLogFileIndex( - filesToCompact.last.getPath.getName))) - } + if (minScore < compactionThresholdScore) { + (CompactionResult.LOW_SCORE_FOR_COMPACTION, None) + } else { + val rewriter = new FilteredEventLogFileRewriter(sparkConf, hadoopConf, fs, filters) + rewriter.rewrite(filesToCompact) + cleanupCompactedFiles(filesToCompact) + (CompactionResult.SUCCESS, Some(RollingEventLogFilesWriter.getEventLogFileIndex( + filesToCompact.last.getPath.getName))) + } + } + + private def loadFilesToFilterBuilder(files: Seq[FileStatus]): Seq[EventFilterBuilder] = { + try { + filterBuildersLoader.loadNewFiles(files) } catch { case _: LowerIndexLoadRequested => // reset loader and load again filterBuildersLoader = new EventFilterBuildersLoader(fs) - doCompact(reader) + loadFilesToFilterBuilder(files) + + case NonFatal(e) => + // reset loader before throwing exception, as filter builders aren't properly loaded + filterBuildersLoader = new EventFilterBuildersLoader(fs) + throw e } } 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 f0466b8ad8f55..88a8ffa548b72 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 @@ -218,15 +218,23 @@ class RollingEventLogFilesFileReader( private lazy val appStatusFile = files.find(isAppStatusFile).get private lazy val eventLogFiles: Seq[FileStatus] = { - val eventLogFiles = files.filter(isEventLogFile).sortBy { status => + val idxToEventLogFiles = files.map { status => val filePath = status.getPath - var idx = getEventLogFileIndex(filePath.getName).toDouble + if (isEventLogFile(filePath.getName)) { + getEventLogFileIndex(filePath.getName) -> status + } else { + -1L -> status + } + }.filter { case (idx, _) => idx >= 0 } + + val eventLogFiles = idxToEventLogFiles.sortBy { case (idx, status) => // trick to place compacted file later than normal file if index is same. - if (EventLogFileWriter.isCompacted(filePath)) { - idx += 0.1 + if (EventLogFileWriter.isCompacted(status.getPath)) { + idx + 0.1 + } else { + idx } - idx - } + }.map(_._2) val filesToRead = dropBeforeLastCompactFile(eventLogFiles) val indices = filesToRead.map { file => getEventLogFileIndex(file.getPath.getName) }