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_r357468359
 
 

 ##########
 File path: 
core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala
 ##########
 @@ -0,0 +1,165 @@
+/*
+ * 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._
+
+/**
+ * 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, 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
+  }
+
+  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 = {
+    _stageToRDDs.getOrElseUpdate(stageSubmitted.stageInfo.stageId,
+      stageSubmitted.stageInfo.rddInfos.map(_.id))
+  }
+
+  override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = {
+    val curTasks = _stageToTasks.getOrElseUpdate(taskStart.stageId,
 
 Review comment:
   Honestly I'm not sure how much the events can be out of order. The code 
assumes job start event will be placed earlier than any stage/task events in 
the job, and job end event will be placed later than any stage/task events in 
the job. If that's not true, may need to revisit the whole builder logic.

----------------------------------------------------------------
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:
[email protected]


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to