From 1bc17a6b8add02772a8a0a1048ac6a01d045baf4 Mon Sep 17 00:00:00 2001 From: Taaffy <32072374+Taaffy@users.noreply.github.com> Date: Tue, 19 Sep 2017 10:20:04 +0100 Subject: [PATCH] [SPARK-22052] Incorrect Metric assigned in MetricsReporter.scala Current implementation for processingRate-total uses wrong metric: mistakenly uses inputRowsPerSecond instead of processedRowsPerSecond ## What changes were proposed in this pull request? Adjust processingRate-total from using inputRowsPerSecond to processedRowsPerSecond ## How was this patch tested? Built spark from source with proposed change and tested output with correct parameter. Before change the csv metrics file for inputRate-total and processingRate-total displayed the same values due to the error. After changing MetricsReporter.scala the processingRate-total csv file displayed the correct metric. processed rows per second Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Taaffy <32072374+Taaffy@users.noreply.github.com> Closes #19268 from Taaffy/patch-1. --- .../apache/spark/sql/execution/streaming/MetricsReporter.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetricsReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetricsReporter.scala index 5551d12fa8ad2..b84e6ce64c611 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetricsReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetricsReporter.scala @@ -40,7 +40,7 @@ class MetricsReporter( // 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", () => stream.lastProgress.inputRowsPerSecond) - registerGauge("processingRate-total", () => stream.lastProgress.inputRowsPerSecond) + registerGauge("processingRate-total", () => stream.lastProgress.processedRowsPerSecond) registerGauge("latency", () => stream.lastProgress.durationMs.get("triggerExecution").longValue()) private def registerGauge[T](name: String, f: () => T)(implicit num: Numeric[T]): Unit = {