Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/15307#discussion_r83082993
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
 ---
    @@ -516,12 +568,127 @@ class StreamExecution(
          """.stripMargin
       }
     
    -  private def toInfo: StreamingQueryInfo = {
    -    new StreamingQueryInfo(
    -      this.name,
    -      this.id,
    -      this.sourceStatuses,
    -      this.sinkStatus)
    +  /**
    +   * Report row metrics of the executed trigger
    +   * @param triggerExecutionPlan Execution plan of the trigger
    +   * @param triggerLogicalPlan Logical plan of the trigger, generated from 
the query logical plan
    +   * @param sourceToDF Source to DataFrame returned by the source.getBatch 
in this trigger
    +   */
    +  private def reportNumRows(
    +      triggerExecutionPlan: SparkPlan,
    +      triggerLogicalPlan: LogicalPlan,
    +      sourceToDF: Map[Source, DataFrame]): Unit = {
    +    // We want to associate execution plan leaves to sources that generate 
them, so that we match
    +    // the their metrics (e.g. numOutputRows) to the sources. To do this 
we do the following.
    +    // Consider the translation from the streaming logical plan to the 
final executed plan.
    +    //
    +    //  streaming logical plan (with sources) <==> trigger's logical plan 
<==> executed plan
    +    //
    +    // 1. We keep track of streaming sources associated with each leaf in 
the trigger's logical plan
    +    //    - Each logical plan leaf will be associated with a single 
streaming source.
    +    //    - There can be multiple logical plan leaves associated with a 
streaming source.
    +    //    - There can be leaves not associated with any streaming source, 
because they were
    +    //      generated from a batch source (e.g. stream-batch joins)
    +    //
    +    // 2. Assuming that the executed plan has same number of leaves in the 
same order as that of
    +    //    the trigger logical plan, we associate executed plan leaves with 
corresponding
    +    //    streaming sources.
    +    //
    +    // 3. For each source, we sum the metrics of the associated execution 
plan leaves.
    +    //
    +    val logicalPlanLeafToSource = sourceToDF.flatMap { case (source, df) =>
    +      df.logicalPlan.collectLeaves().map { leaf => leaf -> source }
    +    }
    +    val allLogicalPlanLeaves = triggerLogicalPlan.collectLeaves() // 
includes non-streaming sources
    +    val allExecPlanLeaves = triggerExecutionPlan.collectLeaves()
    +    val sourceToNumInputRows: Map[Source, Long] =
    +      if (allLogicalPlanLeaves.size == allExecPlanLeaves.size) {
    +        val execLeafToSource = 
allLogicalPlanLeaves.zip(allExecPlanLeaves).flatMap {
    +          case (lp, ep) => logicalPlanLeafToSource.get(lp).map { source => 
ep -> source }
    +        }
    +        val sourceToNumInputRows = execLeafToSource.map { case (execLeaf, 
source) =>
    +          val numRows = 
execLeaf.metrics.get("numOutputRows").map(_.value).getOrElse(0L)
    +          source -> numRows
    +        }
    +        sourceToNumInputRows.groupBy(_._1).mapValues(_.map(_._2).sum) // 
sum up rows for each source
    +      } else {
    +        if (!metricWarningLogged) {
    +          def toString[T](seq: Seq[T]): String = s"(size = ${seq.size}), 
${seq.mkString(", ")}"
    +          logWarning(
    +            "Could not report metrics as number leaves in trigger logical 
plan did not match that" +
    +              s" of the execution plan:\n" +
    +              s"logical plan leaves: ${toString(allLogicalPlanLeaves)}\n" +
    +              s"execution plan leaves: ${toString(allExecPlanLeaves)}\n")
    +          metricWarningLogged = true
    +        }
    +        Map.empty
    +      }
    +    val numOutputRows = 
triggerExecutionPlan.metrics.get("numOutputRows").map(_.value)
    +    val stateNodes = triggerExecutionPlan.collect {
    +      case p if p.isInstanceOf[StateStoreSaveExec] => p
    +    }
    +
    +    streamMetrics.reportNumInputRows(sourceToNumInputRows)
    +    stateNodes.zipWithIndex.foreach { case (s, i) =>
    +      streamMetrics.reportTriggerStatus(
    +        NUM_TOTAL_STATE_ROWS(i + 1),
    +        s.metrics.get("numTotalStateRows").map(_.value).getOrElse(0L))
    +      streamMetrics.reportTriggerStatus(
    +        NUM_UPDATED_STATE_ROWS(i + 1),
    +        s.metrics.get("numUpdatedStateRows").map(_.value).getOrElse(0L))
    +    }
    +    updateStatus()
    +  }
    +
    +  private def reportTimeTaken[T](triggerStatusKey: String)(body: => T): T 
= {
    +    val startTime = triggerClock.getTimeMillis()
    +    val result = body
    +    val endTime = triggerClock.getTimeMillis()
    +    streamMetrics.reportTriggerStatus(triggerStatusKey, math.max(endTime - 
startTime, 0))
    +    updateStatus()
    +    result
    +  }
    +
    +  private def reportTimeTaken[T](source: Source, triggerStatusKey: 
String)(body: => T): T = {
    +    val startTime = triggerClock.getTimeMillis()
    +    val result = body
    +    val endTime = triggerClock.getTimeMillis()
    +    streamMetrics.reportSourceTriggerStatus(
    +      source, triggerStatusKey, math.max(endTime - startTime, 0))
    +    updateStatus()
    +    result
    +  }
    +
    +  private def reportTimestamp(triggerStatusKey: String): Unit = {
    +    streamMetrics.reportTriggerStatus(triggerStatusKey, 
triggerClock.getTimeMillis)
    +    updateStatus()
    +  }
    --- End diff --
    
    What decides when these utility functions are defined here vs in the 
`StreamMetrics` class?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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

Reply via email to