Skip to content

Commit afe083f

Browse files
authored
Incorrect Metric reported in MetricsReporter.scala
Current implementation for processingRate-total uses wrong metric: mistakenly uses inputRowsPerSecond instead of processedRowsPerSecond
1 parent 3b049ab commit afe083f

File tree

1 file changed

+1
-1
lines changed

1 file changed

+1
-1
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetricsReporter.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,7 @@ class MetricsReporter(
4040
// Metric names should not have . in them, so that all the metrics of a query are identified
4141
// together in Ganglia as a single metric group
4242
registerGauge("inputRate-total", () => stream.lastProgress.inputRowsPerSecond)
43-
registerGauge("processingRate-total", () => stream.lastProgress.inputRowsPerSecond)
43+
registerGauge("processingRate-total", () => stream.lastProgress.processedRowsPerSecond)
4444
registerGauge("latency", () => stream.lastProgress.durationMs.get("triggerExecution").longValue())
4545

4646
private def registerGauge[T](name: String, f: () => T)(implicit num: Numeric[T]): Unit = {

0 commit comments

Comments
 (0)