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

    https://github.com/apache/spark/pull/15307#discussion_r82909681
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala
 ---
    @@ -0,0 +1,244 @@
    +/*
    + * 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.streaming
    +
    +import java.{util => ju}
    +
    +import scala.collection.mutable
    +
    +import com.codahale.metrics.{Gauge, MetricRegistry}
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.metrics.source.{Source => CodahaleSource}
    +import org.apache.spark.util.Clock
    +
    +/**
    + * Class that manages all the metrics related to a StreamingQuery. It does 
the following.
    + * - Calculates metrics (rates, latencies, etc.) based on information 
reported by StreamExecution.
    + * - Allows the current metric values to be queried
    + * - Serves some of the metrics through Codahale/DropWizard metrics
    + *
    + * @param sources Unique set of sources in a query
    + * @param triggerClock Clock used for triggering in StreamExecution
    + * @param codahaleSourceName Root name for all the Codahale metrics
    + */
    +class StreamMetrics(sources: Set[Source], triggerClock: Clock, 
codahaleSourceName: String)
    +  extends CodahaleSource with Logging {
    +
    +  import StreamMetrics._
    +
    +  // Trigger infos
    +  private val triggerStatus = new mutable.HashMap[String, String]
    +  private val sourceTriggerStatus = new mutable.HashMap[Source, 
mutable.HashMap[String, String]]
    +
    +  // Rate estimators for sources and sinks
    +  private val inputRates = new mutable.HashMap[Source, RateCalculator]
    +  private val processingRates = new mutable.HashMap[Source, RateCalculator]
    +
    +  // Number of input rows in the current trigger
    +  private val numInputRows = new mutable.HashMap[Source, Long]
    +  private var numOutputRows: Option[Long] = None
    +  private var currentTriggerStartTimestamp: Long = -1
    +  private var previousTriggerStartTimestamp: Long = -1
    +  private var latency: Option[Double] = None
    +
    +  override val sourceName: String = codahaleSourceName
    +  override val metricRegistry: MetricRegistry = new MetricRegistry
    +
    +  // =========== Initialization ===========
    +
    +  // Metric names should not have . in them, so that all the metrics of a 
query are identified
    +  // together in Ganglia as a single metric group
    +  registerGauge("inputRate-total", currentInputRate)
    +  registerGauge("processingRate-total", () => currentProcessingRate)
    +  registerGauge("latency", () => currentLatency().getOrElse(-1.0))
    +
    +  sources.foreach { s =>
    +    inputRates.put(s, new RateCalculator)
    +    processingRates.put(s, new RateCalculator)
    +    sourceTriggerStatus.put(s, new mutable.HashMap[String, String])
    +
    +    registerGauge(s"inputRate-${s.toString}", () => 
currentSourceInputRate(s))
    +    registerGauge(s"processingRate-${s.toString}", () => 
currentSourceProcessingRate(s))
    +  }
    +
    +  // =========== Setter methods ===========
    +
    +  def reportTriggerStarted(triggerId: Long): Unit = synchronized {
    +    numInputRows.clear()
    +    numOutputRows = None
    +    triggerStatus.clear()
    +    sourceTriggerStatus.values.foreach(_.clear())
    +
    +    reportTriggerStatus(TRIGGER_ID, triggerId)
    +    sources.foreach(s => reportSourceTriggerStatus(s, TRIGGER_ID, 
triggerId))
    +    reportTriggerStatus(ACTIVE, true)
    +    currentTriggerStartTimestamp = triggerClock.getTimeMillis()
    +    reportTriggerStatus(START_TIMESTAMP, currentTriggerStartTimestamp)
    +  }
    +
    +  def reportTriggerStatus[T](key: String, value: T): Unit = synchronized {
    +    triggerStatus.put(key, value.toString)
    +  }
    +
    +  def reportSourceTriggerStatus[T](source: Source, key: String, value: T): 
Unit = synchronized {
    +    sourceTriggerStatus(source).put(key, value.toString)
    +  }
    +
    +  def reportNumInputRows(inputRows: Map[Source, Long]): Unit = 
synchronized {
    +    numInputRows ++= inputRows
    +  }
    +
    +  def reportTriggerFinished(): Unit = synchronized {
    +    require(currentTriggerStartTimestamp >= 0)
    +    val currentTriggerFinishTimestamp = triggerClock.getTimeMillis()
    +    reportTriggerStatus(FINISH_TIMESTAMP, currentTriggerFinishTimestamp)
    +    reportTriggerStatus(STATUS_MESSAGE, "")
    +    reportTriggerStatus(ACTIVE, false)
    +
    +    // Report number of rows
    +    val totalNumInputRows = numInputRows.values.sum
    +    reportTriggerStatus(NUM_INPUT_ROWS, totalNumInputRows)
    +    reportTriggerStatus(NUM_OUTPUT_ROWS, numOutputRows.getOrElse(0))
    +    numInputRows.foreach { case (s, r) =>
    +      reportSourceTriggerStatus(s, NUM_SOURCE_INPUT_ROWS, r)
    +    }
    +
    +    val currentTriggerDuration = currentTriggerFinishTimestamp - 
currentTriggerStartTimestamp
    +    val previousInputIntervalOption = if (previousTriggerStartTimestamp >= 
0) {
    +      Some(currentTriggerStartTimestamp - previousTriggerStartTimestamp)
    +    } else None
    +
    +    // Update input rate = num rows received by each source during the 
previous trigger interval
    +    // Interval is measures as interval between start times of previous 
and current trigger.
    +    //
    +    // TODO: Instead of trigger start, we should use time when getOffset 
was called on each source
    +    // as this may be different for each source if there are many sources 
in the query plan
    +    // and getOffset is called serially on them.
    +    if (previousInputIntervalOption.nonEmpty) {
    +      sources.foreach { s =>
    +        inputRates(s).update(numInputRows.getOrElse(s, 0), 
previousInputIntervalOption.get)
    +      }
    +    }
    +
    +    // Update processing rate = num rows processed for each source in 
current trigger duration
    +    sources.foreach { s =>
    +      processingRates(s).update(numInputRows.getOrElse(s, 0), 
currentTriggerDuration)
    +    }
    +
    +    // Update latency = if data present, 0.5 * previous trigger interval + 
current trigger duration
    +    if (previousInputIntervalOption.nonEmpty && totalNumInputRows > 0) {
    +      latency = Some((previousInputIntervalOption.get.toDouble / 2) + 
currentTriggerDuration)
    +    } else {
    +      latency = None
    +    }
    +
    +    previousTriggerStartTimestamp = currentTriggerStartTimestamp
    +    currentTriggerStartTimestamp = -1
    +  }
    +
    +  // =========== Getter methods ===========
    +
    +  def currentInputRate(): Double = synchronized {
    +    // Since we are calculating source input rates using the same time 
interval for all sources
    +    // it is fine to calculate total input rate as the sum of per source 
input rate.
    +    inputRates.map(_._2.currentRate).sum
    +  }
    +
    +  def currentSourceInputRate(source: Source): Double = synchronized {
    +    inputRates(source).currentRate
    +  }
    +
    +  def currentProcessingRate(): Double = synchronized {
    +    // Since we are calculating source processing rates using the same 
time interval for all sources
    +    // it is fine to calculate total processing rate as the sum of per 
source processing rate.
    +    processingRates.map(_._2.currentRate).sum
    +  }
    +
    +  def currentSourceProcessingRate(source: Source): Double = synchronized {
    +    processingRates(source).currentRate
    +  }
    +
    +  def currentLatency(): Option[Double] = synchronized { latency }
    +
    +  def currentTriggerStatus(): Map[String, String] = synchronized { 
triggerStatus.toMap }
    +
    +  def currentSourceTriggerStatus(source: Source): Map[String, String] = 
synchronized {
    +    sourceTriggerStatus(source).toMap
    +  }
    +
    +  // =========== Other methods ===========
    +
    +  private def registerGauge[T](name: String, f: () => T)(implicit num: 
Numeric[T]): Unit = {
    +    synchronized {
    +      metricRegistry.register(name, new Gauge[T] {
    +        override def getValue: T = f()
    +      })
    +    }
    +  }
    +
    +  def stop(): Unit = synchronized {
    +    inputRates.valuesIterator.foreach { _.stop() }
    +    processingRates.valuesIterator.foreach { _.stop() }
    +    latency = None
    +  }
    +}
    +
    +object StreamMetrics extends Logging {
    +  /** Simple utility class to calculate rate while avoiding DivideByZero */
    +  class RateCalculator {
    +    @volatile private var rate: Option[Double] = None
    +
    +    def update(numRows: Long, timeGapMs: Long): Unit = {
    +      if (timeGapMs > 0) {
    +        rate = Some(numRows.toDouble * 1000 / timeGapMs)
    +      } else {
    +        rate = None
    +        logDebug(s"Rate updates cannot with zero or negative time gap 
$timeGapMs")
    +      }
    +    }
    +
    +    def currentRate: Double = rate.getOrElse(0.0)
    +
    +    def stop(): Unit = { rate = None }
    +  }
    +
    +
    +  val TRIGGER_ID = "triggerId"
    +  val ACTIVE = "isActive"
    +  val DATA_AVAILABLE = "isDataAvailable"
    +  val STATUS_MESSAGE = "statusMessage"
    +
    +  val START_TIMESTAMP = "timestamp.triggerStart"
    +  val GET_OFFSET_TIMESTAMP = "timestamp.afterGetOffset"
    +  val GET_BATCH_TIMESTAMP = "timestamp.afterGetBatch"
    +  val FINISH_TIMESTAMP = "timestamp.triggerFinish"
    +
    +  val GET_OFFSET_LATENCY = "latency.getOffset"
    +  val OFFSET_WAL_WRITE_LATENCY = "latency.offsetLogWrite"
    +  val GET_BATCH_LATENCY = "latency.getBatch"
    +  val TRIGGER_LATENCY = "latency.fullTrigger"
    +  val SOURCE_GET_OFFSET_LATENCY = "latency.sourceGetOffset"
    +  val SOURCE_GET_BATCH_LATENCY = "latency.sourceGetBatch"
    +
    +  val NUM_INPUT_ROWS = "numRows.input.total"
    +  val NUM_OUTPUT_ROWS = "numRows.output"
    +  val NUM_SOURCE_INPUT_ROWS = "numRows.input.source"
    +  def NUM_TOTAL_STATE_ROWS(aggId: Int): String = 
s"numRows.state.aggregation$aggId.total"
    --- End diff --
    
    I added a sqlconf to disable it by default, can be enabled by setting that 
metric when needed for production uses. 


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