Skip to content

Commit

Permalink
[SPARK-29779][CORE] Compact old event log files and cleanup
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?

This patch proposes to compact old event log files when end users enable rolling event log, and clean up these files after compaction.

Here the "compaction" really mean is filtering out listener events for finished/removed things - like jobs which take most of space for event log file except SQL related events. To achieve this, compactor does two phases reading: 1) tracking the live jobs (and more to add) 2) filtering events via leveraging the information about live things and rewriting to the "compacted" file.

This approach retains the ability of compatibility on event log file and adds the possibility of reducing the overall size of event logs. There's a downside here as well: executor metrics for tasks would be inaccurate, as compactor will filter out the task events which job is finished, but I don't feel it as a blocker.

Please note that SPARK-29779 leaves below functionalities for future JIRA issue as the patch for SPARK-29779 is too huge and we decided to break down:

* apply filter in SQL events
* integrate compaction into FsHistoryProvider
* documentation about new configuration

### Why are the changes needed?

One of major goal of SPARK-28594 is to prevent the event logs to become too huge, and SPARK-29779 achieves the goal. We've got another approach in prior, but the old approach required models in both KVStore and live entities to guarantee compatibility, while they're not designed to do so.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Added UTs.

Closes #27085 from HeartSaVioR/SPARK-29779-part1.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
  • Loading branch information
HeartSaVioR authored and Marcelo Vanzin committed Jan 10, 2020
1 parent 2bd8731 commit 7fb17f5
Show file tree
Hide file tree
Showing 15 changed files with 1,545 additions and 58 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
org.apache.spark.deploy.history.BasicEventFilterBuilder
Original file line number Diff line number Diff line change
@@ -0,0 +1,176 @@
/*
* 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.SparkContext
import org.apache.spark.deploy.history.EventFilter.FilterStatistics
import org.apache.spark.internal.Logging
import org.apache.spark.scheduler._
import org.apache.spark.storage.BlockManagerId

/**
* This class tracks both live jobs and live executors, and pass the list to the
* [[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, Set[Int]]
private val _stageToTasks = new mutable.HashMap[Int, mutable.Set[Long]]
private val _stageToRDDs = new mutable.HashMap[Int, Set[Int]]
private val _liveExecutors = new mutable.HashSet[String]

private var totalJobs: Long = 0L
private var totalStages: Long = 0L
private var totalTasks: Long = 0L

def liveJobs: Set[Int] = _liveJobToStages.keySet.toSet
def liveStages: Set[Int] = _stageToRDDs.keySet.toSet
def liveTasks: Set[Long] = _stageToTasks.values.flatten.toSet
def liveRDDs: Set[Int] = _stageToRDDs.values.flatten.toSet
def liveExecutors: Set[String] = _liveExecutors.toSet

override def onJobStart(jobStart: SparkListenerJobStart): Unit = {
totalJobs += 1
totalStages += jobStart.stageIds.length
_liveJobToStages += jobStart.jobId -> jobStart.stageIds.toSet
}

override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = {
val stages = _liveJobToStages.getOrElse(jobEnd.jobId, Seq.empty[Int])
_liveJobToStages -= jobEnd.jobId
_stageToTasks --= stages
_stageToRDDs --= stages
}

override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = {
val stageId = stageSubmitted.stageInfo.stageId
_stageToRDDs.put(stageId, stageSubmitted.stageInfo.rddInfos.map(_.id).toSet)
_stageToTasks.getOrElseUpdate(stageId, new mutable.HashSet[Long]())
}

override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = {
totalTasks += 1
_stageToTasks.get(taskStart.stageId).foreach { tasks =>
tasks += 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 = {
val stats = FilterStatistics(totalJobs, liveJobs.size, totalStages,
liveStages.size, totalTasks, liveTasks.size)

new BasicEventFilter(stats, liveJobs, liveStages, liveTasks, liveRDDs, liveExecutors)
}
}

/**
* 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(
stats: Option[FilterStatistics],
liveJobs: Set[Int],
liveStages: Set[Int],
liveTasks: Set[Long],
liveRDDs: Set[Int]) extends EventFilter with Logging {

logDebug(s"jobs : $liveJobs")
logDebug(s"stages : $liveStages")
logDebug(s"tasks : $liveTasks")
logDebug(s"RDDs : $liveRDDs")

override def statistics(): Option[FilterStatistics] = stats

protected val acceptFnForJobEvents: PartialFunction[SparkListenerEvent, Boolean] = {
case e: SparkListenerStageCompleted =>
liveStages.contains(e.stageInfo.stageId)
case e: SparkListenerStageSubmitted =>
liveStages.contains(e.stageInfo.stageId)
case e: SparkListenerTaskStart =>
liveTasks.contains(e.taskInfo.taskId)
case e: SparkListenerTaskGettingResult =>
liveTasks.contains(e.taskInfo.taskId)
case e: SparkListenerTaskEnd =>
liveTasks.contains(e.taskInfo.taskId)
case e: SparkListenerJobStart =>
liveJobs.contains(e.jobId)
case e: SparkListenerJobEnd =>
liveJobs.contains(e.jobId)
case e: SparkListenerUnpersistRDD =>
liveRDDs.contains(e.rddId)
case e: SparkListenerExecutorMetricsUpdate =>
e.accumUpdates.exists { case (taskId, stageId, _, _) =>
liveTasks.contains(taskId) || liveStages.contains(stageId)
}
case e: SparkListenerSpeculativeTaskSubmitted =>
liveStages.contains(e.stageId)
}
}

/**
* 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".
*/
private[spark] class BasicEventFilter(
_stats: FilterStatistics,
_liveJobs: Set[Int],
_liveStages: Set[Int],
_liveTasks: Set[Long],
_liveRDDs: Set[Int],
liveExecutors: Set[String])
extends JobEventFilter(
Some(_stats),
_liveJobs,
_liveStages,
_liveTasks,
_liveRDDs) with Logging {

logDebug(s"live executors : $liveExecutors")

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)
case e: SparkListenerBlockManagerAdded => acceptBlockManagerEvent(e.blockManagerId)
case e: SparkListenerBlockManagerRemoved => acceptBlockManagerEvent(e.blockManagerId)
case e: SparkListenerBlockUpdated => acceptBlockManagerEvent(e.blockUpdatedInfo.blockManagerId)
}

private def acceptBlockManagerEvent(blockManagerId: BlockManagerId): Boolean = {
blockManagerId.isDriver || liveExecutors.contains(blockManagerId.executorId)
}

override def acceptFn(): PartialFunction[SparkListenerEvent, Boolean] = {
_acceptFn.orElse(acceptFnForJobEvents)
}
}
109 changes: 109 additions & 0 deletions core/src/main/scala/org/apache/spark/deploy/history/EventFilter.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
/*
* 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.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.FilterStatistics
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
* by [[SparkListenerInterface]], and create a new [[EventFilter]] instance which leverages
* information gathered to decide whether the event should be accepted or not.
*/
private[spark] trait EventFilterBuilder extends SparkListenerInterface {
def createFilter(): EventFilter
}

/** [[EventFilter]] decides whether the given event should be accepted or rejected. */
private[spark] trait EventFilter {
/**
* 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 statistics(): Option[FilterStatistics]

/**
* Classify whether the event is accepted or rejected by this filter.
*
* 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 acceptFn(): PartialFunction[SparkListenerEvent, Boolean]
}

private[spark] object EventFilter extends Logging {
case class FilterStatistics(
totalJobs: Long,
liveJobs: Long,
totalStages: Long,
liveStages: Long,
totalTasks: Long,
liveTasks: Long)

def applyFilterToFile(
fs: FileSystem,
filters: Seq[EventFilter],
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()

lines.zipWithIndex.foreach { case (line, lineNum) =>
try {
val event = try {
Some(JsonProtocol.sparkEventFromJson(parse(line)))
} catch {
// ignore any exception occurred from unidentified json
case NonFatal(_) =>
onUnidentified(line)
None
}

event.foreach { e =>
val results = filters.flatMap(_.acceptFn().lift.apply(e))
if (results.nonEmpty && results.forall(_ == false)) {
onRejected(line, e)
} else {
onAccepted(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
}
}
}
}
}
Loading

0 comments on commit 7fb17f5

Please sign in to comment.