Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,13 @@

package org.apache.spark.sql.execution.streaming

import java.text.SimpleDateFormat

import com.codahale.metrics.{Gauge, MetricRegistry}

import org.apache.spark.internal.Logging
import org.apache.spark.metrics.source.{Source => CodahaleSource}
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.streaming.StreamingQueryProgress

/**
Expand All @@ -39,6 +42,23 @@ class MetricsReporter(
registerGauge("processingRate-total", _.processedRowsPerSecond, 0.0)
registerGauge("latency", _.durationMs.get("triggerExecution").longValue(), 0L)

private val timestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601
timestampFormat.setTimeZone(DateTimeUtils.getTimeZone("UTC"))

registerGauge("eventTime-watermark",
progress => convertStringDateToMillis(progress.eventTime.get("watermark")), 0L)

registerGauge("states-rowsTotal", _.stateOperators.map(_.numRowsTotal).sum, 0L)
registerGauge("states-usedBytes", _.stateOperators.map(_.memoryUsedBytes).sum, 0L)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can add more metrics like "providerLoadedMapSizeBytes" after adopting SPARK-24441, so that actual memory usage of state store provider could be tracked via time-series manner.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Those are custom metrics, which may or may not be present depending on the implementation of state store. I dont recommend adding them here directly.

Copy link
Contributor Author

@HeartSaVioR HeartSaVioR Aug 1, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the input! I'll keep the patch as it is.

Could you suggest some approaches to extend the maintained metrics? I would like to expand more, and newer things might be coming from custom metrics (like from source and sink) so might be worth to have extension point.

My question is beyond of this PR, so please continue reviewing the patch. Thanks!

private def convertStringDateToMillis(isoUtcDateStr: String) = {
if (isoUtcDateStr != null) {
timestampFormat.parse(isoUtcDateStr).getTime
} else {
0L
}
}

private def registerGauge[T](
name: String,
f: StreamingQueryProgress => T,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -462,6 +462,9 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi
assert(gauges.get("latency").getValue.asInstanceOf[Long] == 0)
assert(gauges.get("processingRate-total").getValue.asInstanceOf[Double] == 0.0)
assert(gauges.get("inputRate-total").getValue.asInstanceOf[Double] == 0.0)
assert(gauges.get("eventTime-watermark").getValue.asInstanceOf[Long] == 0)
assert(gauges.get("states-rowsTotal").getValue.asInstanceOf[Long] == 0)
assert(gauges.get("states-usedBytes").getValue.asInstanceOf[Long] == 0)
sq.stop()
}
}
Expand Down