vanzin 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_r355805440
 
 

 ##########
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.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.sql.execution.history
+
+import scala.collection.mutable
+
+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
+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 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.
+ */
+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)
+    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
+    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)
+      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]())
 
 Review comment:
   Could you use `_liveExecutionToJobs.remove` and the everything past L95 
could be inside `jobs.foreach`?

----------------------------------------------------------------
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