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

    https://github.com/apache/spark/pull/15307#discussion_r83082403
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala
 ---
    @@ -0,0 +1,240 @@
    +/*
    + * 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 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()
    +    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)
    --- End diff --
    
    What is the meaning of `ACTIVE`?  It seems it is different than `state == 
ACTIVE`?


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