@@ -37,10 +37,17 @@ private[ui] class StreamingUIListener(ssc: StreamingContext) extends StreamingLi
37
37
private val runningBatchInfos = new HashMap [Time , BatchInfo ]
38
38
private val completedaBatchInfos = new Queue [BatchInfo ]
39
39
private val batchInfoLimit = ssc.conf.getInt(" spark.steaming.ui.maxBatches" , 100 )
40
- private var totalBatchesCompleted = 0L
40
+ private var totalCompletedBatches = 0L
41
+ private val receiverInfos = new HashMap [Int , ReceiverInfo ]
41
42
42
43
val batchDuration = ssc.graph.batchDuration.milliseconds
43
44
45
+ override def onReceiverStarted (receiverStarted : StreamingListenerReceiverStarted ) = {
46
+ synchronized {
47
+ receiverInfos.put(receiverStarted.receiverInfo.streamId, receiverStarted.receiverInfo)
48
+ }
49
+ }
50
+
44
51
override def onBatchSubmitted (batchSubmitted : StreamingListenerBatchSubmitted ) = synchronized {
45
52
runningBatchInfos(batchSubmitted.batchInfo.batchTime) = batchSubmitted.batchInfo
46
53
}
@@ -55,15 +62,19 @@ private[ui] class StreamingUIListener(ssc: StreamingContext) extends StreamingLi
55
62
runningBatchInfos.remove(batchCompleted.batchInfo.batchTime)
56
63
completedaBatchInfos.enqueue(batchCompleted.batchInfo)
57
64
if (completedaBatchInfos.size > batchInfoLimit) completedaBatchInfos.dequeue()
58
- totalBatchesCompleted += 1L
65
+ totalCompletedBatches += 1L
59
66
}
60
67
61
- def numTotalBatchesCompleted : Long = synchronized {
62
- totalBatchesCompleted
68
+ def numNetworkReceivers = synchronized {
69
+ ssc.graph.getNetworkInputStreams().size
63
70
}
64
71
65
- def numNetworkReceivers : Int = synchronized {
66
- completedaBatchInfos.headOption.map(_.receivedBlockInfo.size).getOrElse(0 )
72
+ def numTotalCompletedBatches : Long = synchronized {
73
+ totalCompletedBatches
74
+ }
75
+
76
+ def numUnprocessedBatches : Long = synchronized {
77
+ waitingBatchInfos.size + runningBatchInfos.size
67
78
}
68
79
69
80
def waitingBatches : Seq [BatchInfo ] = synchronized {
@@ -91,9 +102,7 @@ private[ui] class StreamingUIListener(ssc: StreamingContext) extends StreamingLi
91
102
}
92
103
93
104
def receivedRecordsDistributions : Map [Int , Option [Distribution ]] = synchronized {
94
- val allBatcheInfos = waitingBatchInfos.values.toSeq ++
95
- runningBatchInfos.values.toSeq ++ completedaBatchInfos
96
- val latestBatchInfos = allBatcheInfos.sortBy(_.batchTime)(Time .ordering).reverse.take(batchInfoLimit)
105
+ val latestBatchInfos = allBatches.reverse.take(batchInfoLimit)
97
106
val latestBlockInfos = latestBatchInfos.map(_.receivedBlockInfo)
98
107
(0 until numNetworkReceivers).map { receiverId =>
99
108
val blockInfoOfParticularReceiver = latestBlockInfos.map(_.get(receiverId).getOrElse(Array .empty))
@@ -103,6 +112,34 @@ private[ui] class StreamingUIListener(ssc: StreamingContext) extends StreamingLi
103
112
}.toMap
104
113
}
105
114
115
+ def lastReceivedBatchRecords : Map [Int , Long ] = {
116
+ val lastReceivedBlockInfoOption = lastReceivedBatch.map(_.receivedBlockInfo)
117
+ lastReceivedBlockInfoOption.map { lastReceivedBlockInfo =>
118
+ (0 until numNetworkReceivers).map { receiverId =>
119
+ (receiverId, lastReceivedBlockInfo(receiverId).map(_.numRecords).sum)
120
+ }.toMap
121
+ }.getOrElse {
122
+ (0 until numNetworkReceivers).map(receiverId => (receiverId, 0L )).toMap
123
+ }
124
+ }
125
+
126
+ def receiverInfo (receiverId : Int ): Option [ReceiverInfo ] = {
127
+ receiverInfos.get(receiverId)
128
+ }
129
+
130
+ def lastCompletedBatch : Option [BatchInfo ] = {
131
+ completedaBatchInfos.sortBy(_.batchTime)(Time .ordering).lastOption
132
+ }
133
+
134
+ def lastReceivedBatch : Option [BatchInfo ] = {
135
+ allBatches.lastOption
136
+ }
137
+
138
+ private def allBatches : Seq [BatchInfo ] = synchronized {
139
+ (waitingBatchInfos.values.toSeq ++
140
+ runningBatchInfos.values.toSeq ++ completedaBatchInfos).sortBy(_.batchTime)(Time .ordering)
141
+ }
142
+
106
143
private def extractDistribution (getMetric : BatchInfo => Option [Long ]): Option [Distribution ] = {
107
144
Distribution (completedaBatchInfos.flatMap(getMetric(_)).map(_.toDouble))
108
145
}
@@ -114,13 +151,13 @@ private[ui] class StreamingPage(parent: StreamingUI) extends Logging {
114
151
private val listener = parent.listener
115
152
private val calendar = Calendar .getInstance()
116
153
private val startTime = calendar.getTime()
117
-
154
+ private val emptyCellTest = " - "
118
155
119
156
def render (request : HttpServletRequest ): Seq [Node ] = {
120
157
121
158
val content =
122
159
generateBasicStats() ++
123
- <h4 >Statistics over last {listener.completedBatches.size} processed batches</h4 > ++
160
+ <br ></ br >< h4 >Statistics over last {listener.completedBatches.size} processed batches</h4 > ++
124
161
generateNetworkStatsTable() ++
125
162
generateBatchStatsTable()
126
163
UIUtils .headerStreamingPage(content, " " , parent.appName, " Spark Streaming Overview" )
@@ -136,28 +173,76 @@ private[ui] class StreamingPage(parent: StreamingUI) extends Logging {
136
173
<li >
137
174
<strong >Time since start : </strong >{msDurationToString(timeSinceStart)}
138
175
</li >
176
+ <li >
177
+ <strong >Network receivers : </strong >{listener.numNetworkReceivers}
178
+ </li >
139
179
<li >
140
180
<strong >Batch interval : </strong >{msDurationToString(listener.batchDuration)}
141
181
</li >
142
182
<li >
143
- <strong >Processed batches : </strong >{listener.numTotalBatchesCompleted}
183
+ <strong >Processed batches : </strong >{listener.numTotalCompletedBatches}
184
+ </li >
185
+ <li >
186
+ <strong >Waiting batches : </strong >{listener.numUnprocessedBatches}
144
187
</li >
145
- <li ></li >
146
188
</ul >
147
189
}
148
190
191
+ private def generateNetworkStatsTable (): Seq [Node ] = {
192
+ val receivedRecordDistributions = listener.receivedRecordsDistributions
193
+ val lastBatchReceivedRecord = listener.lastReceivedBatchRecords
194
+ val table = if (receivedRecordDistributions.size > 0 ) {
195
+ val headerRow = Seq (
196
+ " Receiver" ,
197
+ " Location" ,
198
+ s " Records in last batch " ,
199
+ " Minimum rate [records/sec]" ,
200
+ " 25th percentile rate [records/sec]" ,
201
+ " Median rate [records/sec]" ,
202
+ " 75th percentile rate [records/sec]" ,
203
+ " Maximum rate [records/sec]"
204
+ )
205
+ val dataRows = (0 until listener.numNetworkReceivers).map { receiverId =>
206
+ val receiverInfo = listener.receiverInfo(receiverId)
207
+ val receiverName = receiverInfo.map(_.toString).getOrElse(s " Receiver- $receiverId" )
208
+ val receiverLocation = receiverInfo.map(_.location).getOrElse(emptyCellTest)
209
+ val receiverLastBatchRecords = numberToString(lastBatchReceivedRecord(receiverId))
210
+ val receivedRecordStats = receivedRecordDistributions(receiverId).map { d =>
211
+ d.getQuantiles().map(r => numberToString(r.toLong))
212
+ }.getOrElse {
213
+ Seq (emptyCellTest, emptyCellTest, emptyCellTest, emptyCellTest, emptyCellTest)
214
+ }
215
+ Seq (receiverName, receiverLocation, receiverLastBatchRecords) ++
216
+ receivedRecordStats
217
+ }
218
+ Some (UIUtils .listingTable(headerRow, dataRows, fixedWidth = true ))
219
+ } else {
220
+ None
221
+ }
222
+
223
+ val content =
224
+ <h5 >Network Input Statistics </h5 > ++
225
+ <div >{table.getOrElse(" No network receivers" )}</div >
226
+
227
+ content
228
+ }
229
+
149
230
private def generateBatchStatsTable (): Seq [Node ] = {
150
231
val numBatches = listener.completedBatches.size
232
+ val lastCompletedBatch = listener.lastCompletedBatch
151
233
val table = if (numBatches > 0 ) {
152
234
val processingDelayQuantilesRow =
153
- " Processing Times" +: getQuantiles(listener.processingDelayDistribution)
235
+ Seq (" Processing Time" , msDurationToString(lastCompletedBatch.flatMap(_.processingDelay))) ++
236
+ getQuantiles(listener.processingDelayDistribution)
154
237
val schedulingDelayQuantilesRow =
155
- " Scheduling Delay:" +: getQuantiles(listener.schedulingDelayDistribution)
238
+ Seq (" Scheduling Delay" , msDurationToString(lastCompletedBatch.flatMap(_.schedulingDelay))) ++
239
+ getQuantiles(listener.schedulingDelayDistribution)
156
240
val totalDelayQuantilesRow =
157
- " End-to-end Delay:" +: getQuantiles(listener.totalDelayDistribution)
241
+ Seq (" Total Delay" , msDurationToString(lastCompletedBatch.flatMap(_.totalDelay))) ++
242
+ getQuantiles(listener.totalDelayDistribution)
158
243
159
- val headerRow = Seq (" Metric" , " Min " , " 25th percentile" ,
160
- " Median" , " 75th percentile" , " Max " )
244
+ val headerRow = Seq (" Metric" , " Last batch " , " Minimum " , " 25th percentile" ,
245
+ " Median" , " 75th percentile" , " Maximum " )
161
246
val dataRows : Seq [Seq [String ]] = Seq (
162
247
processingDelayQuantilesRow,
163
248
schedulingDelayQuantilesRow,
@@ -168,57 +253,19 @@ private[ui] class StreamingPage(parent: StreamingUI) extends Logging {
168
253
None
169
254
}
170
255
171
- val batchCounts =
172
- <ul class =" unstyled" >
173
- <li >
174
- # batches being processed : {listener.runningBatches.size}
175
- </li >
176
- <li >
177
- # scheduled batches : {listener.waitingBatches.size}
178
- </li >
179
- </ul >
180
-
181
- val batchStats =
182
- <ul class =" unstyled" >
183
- {table.getOrElse(" No statistics have been generated yet." )}
184
- </ul >
185
-
186
256
val content =
187
257
<h5 >Batch Processing Statistics </h5 > ++
188
- <div >{batchStats}</div >
189
-
190
- content
191
- }
192
-
193
- private def generateNetworkStatsTable (): Seq [Node ] = {
194
- val receivedRecordDistributions = listener.receivedRecordsDistributions
195
- val numNetworkReceivers = receivedRecordDistributions.size
196
- val table = if (receivedRecordDistributions.size > 0 ) {
197
- val headerRow = Seq (" Receiver" , " Min" , " 25th percentile" ,
198
- " Median" , " 75th percentile" , " Max" )
199
- val dataRows = (0 until numNetworkReceivers).map { receiverId =>
200
- val receiverName = s " Receiver- $receiverId"
201
- val receivedRecordStats = receivedRecordDistributions(receiverId).map { d =>
202
- d.getQuantiles().map(r => numberToString(r.toLong) + " records/second" )
203
- }.getOrElse {
204
- Seq (" -" , " -" , " -" , " -" , " -" )
205
- }
206
- receiverName +: receivedRecordStats
207
- }
208
- Some (UIUtils .listingTable(headerRow, dataRows, fixedWidth = true ))
209
- } else {
210
- None
211
- }
212
-
213
- val content =
214
- <h5 >Network Input Statistics </h5 > ++
215
- <div >{table.getOrElse(" No network receivers" )}</div >
258
+ <div >
259
+ <ul class =" unstyled" >
260
+ {table.getOrElse(" No statistics have been generated yet." )}
261
+ </ul >
262
+ </div >
216
263
217
264
content
218
265
}
219
266
220
267
private def getQuantiles (timeDistributionOption : Option [Distribution ]) = {
221
- timeDistributionOption.get.getQuantiles().map { ms => Utils . msDurationToString(ms.toLong) }
268
+ timeDistributionOption.get.getQuantiles().map { ms => msDurationToString(ms.toLong) }
222
269
}
223
270
224
271
private def numberToString (records : Double ): String = {
@@ -229,13 +276,13 @@ private[ui] class StreamingPage(parent: StreamingUI) extends Logging {
229
276
230
277
val (value, unit) = {
231
278
if (records >= 2 * trillion) {
232
- (records / trillion, " T" )
279
+ (records / trillion, " T" )
233
280
} else if (records >= 2 * billion) {
234
- (records / billion, " B" )
281
+ (records / billion, " B" )
235
282
} else if (records >= 2 * million) {
236
- (records / million, " M" )
283
+ (records / million, " M" )
237
284
} else if (records >= 2 * thousand) {
238
- (records / thousand, " K" )
285
+ (records / thousand, " K" )
239
286
} else {
240
287
(records, " " )
241
288
}
@@ -265,7 +312,7 @@ private[ui] class StreamingPage(parent: StreamingUI) extends Logging {
265
312
}
266
313
}
267
314
268
- val millisecondsString = if (ms % second == 0 ) " " else s " ${ms % second} ms "
315
+ val millisecondsString = if (ms >= second && ms % second == 0 ) " " else s " ${ms % second} ms "
269
316
val secondString = toString((ms % minute) / second, " second" )
270
317
val minuteString = toString((ms % hour) / minute, " minute" )
271
318
val hourString = toString((ms % day) / hour, " hour" )
@@ -292,6 +339,10 @@ private[ui] class StreamingPage(parent: StreamingUI) extends Logging {
292
339
return " "
293
340
}
294
341
}
342
+
343
+ private def msDurationToString (msOption : Option [Long ]): String = {
344
+ msOption.map(msDurationToString).getOrElse(emptyCellTest)
345
+ }
295
346
}
296
347
297
348
0 commit comments