@@ -29,9 +29,17 @@ private[streaming] class StreamingSource(ssc: StreamingContext) extends Source {
2929 private val streamingListener = ssc.progressListener
3030
3131 private def registerGauge [T ](name : String , f : StreamingJobProgressListener => T ,
32- defaultValue : T ) {
32+ defaultValue : T ): Unit = {
33+ registerGaugeWithOption[T ](name,
34+ (l : StreamingJobProgressListener ) => Option (f(streamingListener)), defaultValue)
35+ }
36+
37+ private def registerGaugeWithOption [T ](
38+ name : String ,
39+ f : StreamingJobProgressListener => Option [T ],
40+ defaultValue : T ): Unit = {
3341 metricRegistry.register(MetricRegistry .name(" streaming" , name), new Gauge [T ] {
34- override def getValue : T = Option ( f(streamingListener) ).getOrElse(defaultValue)
42+ override def getValue : T = f(streamingListener).getOrElse(defaultValue)
3543 })
3644 }
3745
@@ -41,6 +49,12 @@ private[streaming] class StreamingSource(ssc: StreamingContext) extends Source {
4149 // Gauge for number of total completed batches
4250 registerGauge(" totalCompletedBatches" , _.numTotalCompletedBatches, 0L )
4351
52+ // Gauge for number of total received records
53+ registerGauge(" totalReceivedRecords" , _.numTotalReceivedRecords, 0L )
54+
55+ // Gauge for number of total processed records
56+ registerGauge(" totalProcessedRecords" , _.numTotalProcessedRecords, 0L )
57+
4458 // Gauge for number of unprocessed batches
4559 registerGauge(" unprocessedBatches" , _.numUnprocessedBatches, 0L )
4660
@@ -55,19 +69,30 @@ private[streaming] class StreamingSource(ssc: StreamingContext) extends Source {
5569
5670 // Gauge for last completed batch, useful for monitoring the streaming job's running status,
5771 // displayed data -1 for any abnormal condition.
58- registerGauge(" lastCompletedBatch_submissionTime" ,
59- _.lastCompletedBatch.map(_.submissionTime).getOrElse(- 1L ), - 1L )
60- registerGauge(" lastCompletedBatch_processStartTime" ,
61- _.lastCompletedBatch.flatMap(_.processingStartTime).getOrElse(- 1L ), - 1L )
62- registerGauge(" lastCompletedBatch_processEndTime" ,
63- _.lastCompletedBatch.flatMap(_.processingEndTime).getOrElse(- 1L ), - 1L )
72+ registerGaugeWithOption(" lastCompletedBatch_submissionTime" ,
73+ _.lastCompletedBatch.map(_.submissionTime), - 1L )
74+ registerGaugeWithOption(" lastCompletedBatch_processingStartTime" ,
75+ _.lastCompletedBatch.flatMap(_.processingStartTime), - 1L )
76+ registerGaugeWithOption(" lastCompletedBatch_processingEndTime" ,
77+ _.lastCompletedBatch.flatMap(_.processingEndTime), - 1L )
78+
79+ // Gauge for last completed batch's delay information.
80+ registerGaugeWithOption(" lastCompletedBatch_processingDelay" ,
81+ _.lastCompletedBatch.flatMap(_.processingDelay), - 1L )
82+ registerGaugeWithOption(" lastCompletedBatch_schedulingDelay" ,
83+ _.lastCompletedBatch.flatMap(_.schedulingDelay), - 1L )
84+ registerGaugeWithOption(" lastCompletedBatch_totalDelay" ,
85+ _.lastCompletedBatch.flatMap(_.totalDelay), - 1L )
6486
6587 // Gauge for last received batch, useful for monitoring the streaming job's running status,
6688 // displayed data -1 for any abnormal condition.
67- registerGauge(" lastReceivedBatch_submissionTime" ,
68- _.lastCompletedBatch.map(_.submissionTime).getOrElse(- 1L ), - 1L )
69- registerGauge(" lastReceivedBatch_processStartTime" ,
70- _.lastCompletedBatch.flatMap(_.processingStartTime).getOrElse(- 1L ), - 1L )
71- registerGauge(" lastReceivedBatch_processEndTime" ,
72- _.lastCompletedBatch.flatMap(_.processingEndTime).getOrElse(- 1L ), - 1L )
89+ registerGaugeWithOption(" lastReceivedBatch_submissionTime" ,
90+ _.lastCompletedBatch.map(_.submissionTime), - 1L )
91+ registerGaugeWithOption(" lastReceivedBatch_processingStartTime" ,
92+ _.lastCompletedBatch.flatMap(_.processingStartTime), - 1L )
93+ registerGaugeWithOption(" lastReceivedBatch_processingEndTime" ,
94+ _.lastCompletedBatch.flatMap(_.processingEndTime), - 1L )
95+
96+ // Gauge for last received batch records.
97+ registerGauge(" lastReceivedBatch_records" , _.lastReceivedBatchRecords.values.sum, 0L )
7398}
0 commit comments