HeartSaVioR commented on a change in pull request #26416: [SPARK-29779][CORE] 
Compact old event log files and cleanup
URL: https://github.com/apache/spark/pull/26416#discussion_r346649250
 
 

 ##########
 File path: 
core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala
 ##########
 @@ -0,0 +1,235 @@
+/*
+ * 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 = eventLogFiles.drop(lastCompactedFileIdx)
+
+    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)
 
 Review comment:
   Please check the UT of FilteredEventLogFileRewriterSuite out, it covers None 
& None.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to