|
17 | 17 |
|
18 | 18 | package org.apache.spark.streaming.ui |
19 | 19 |
|
| 20 | +import java.util.Properties |
| 21 | + |
20 | 22 | import org.scalatest.Matchers |
21 | 23 |
|
| 24 | +import org.apache.spark.scheduler.SparkListenerJobStart |
22 | 25 | import org.apache.spark.streaming.dstream.DStream |
23 | 26 | import org.apache.spark.streaming.scheduler._ |
24 | 27 | import org.apache.spark.streaming.{Duration, Time, Milliseconds, TestSuiteBase} |
@@ -64,6 +67,48 @@ class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { |
64 | 67 | listener.numTotalProcessedRecords should be (0) |
65 | 68 | listener.numTotalReceivedRecords should be (600) |
66 | 69 |
|
| 70 | + // onJobStart |
| 71 | + val properties1 = new Properties() |
| 72 | + properties1.setProperty(JobScheduler.BATCH_TIME_PROPERTY_KEY, Time(1000).milliseconds.toString) |
| 73 | + properties1.setProperty(JobScheduler.OUTPUT_OP_ID_PROPERTY_KEY, 0.toString) |
| 74 | + val jobStart1 = SparkListenerJobStart(jobId = 0, |
| 75 | + 0L, // unused |
| 76 | + Nil, // unused |
| 77 | + properties1) |
| 78 | + listener.onJobStart(jobStart1) |
| 79 | + |
| 80 | + val properties2 = new Properties() |
| 81 | + properties2.setProperty(JobScheduler.BATCH_TIME_PROPERTY_KEY, Time(1000).milliseconds.toString) |
| 82 | + properties2.setProperty(JobScheduler.OUTPUT_OP_ID_PROPERTY_KEY, 0.toString) |
| 83 | + val jobStart2 = SparkListenerJobStart(jobId = 1, |
| 84 | + 0L, // unused |
| 85 | + Nil, // unused |
| 86 | + properties2) |
| 87 | + listener.onJobStart(jobStart2) |
| 88 | + |
| 89 | + val properties3 = new Properties() |
| 90 | + properties3.setProperty(JobScheduler.BATCH_TIME_PROPERTY_KEY, Time(1000).milliseconds.toString) |
| 91 | + properties3.setProperty(JobScheduler.OUTPUT_OP_ID_PROPERTY_KEY, 1.toString) |
| 92 | + val jobStart3 = SparkListenerJobStart(jobId = 0, |
| 93 | + 0L, // unused |
| 94 | + Nil, // unused |
| 95 | + properties3) |
| 96 | + listener.onJobStart(jobStart3) |
| 97 | + |
| 98 | + val properties4 = new Properties() |
| 99 | + properties4.setProperty(JobScheduler.BATCH_TIME_PROPERTY_KEY, Time(1000).milliseconds.toString) |
| 100 | + properties4.setProperty(JobScheduler.OUTPUT_OP_ID_PROPERTY_KEY, 1.toString) |
| 101 | + val jobStart4 = SparkListenerJobStart(jobId = 1, |
| 102 | + 0L, // unused |
| 103 | + Nil, // unused |
| 104 | + properties4) |
| 105 | + listener.onJobStart(jobStart4) |
| 106 | + |
| 107 | + val batchUIData = listener.getBatchUIData(Time(1000)) |
| 108 | + assert(batchUIData != None) |
| 109 | + assert(batchUIData.get.batchInfo === batchInfoStarted) |
| 110 | + assert(batchUIData.get.outputOpIdToSparkJobIds === Seq(0 -> Seq(0, 1), 1 -> Seq(0, 1))) |
| 111 | + |
67 | 112 | // onBatchCompleted |
68 | 113 | val batchInfoCompleted = BatchInfo(Time(1000), receivedBlockInfo, 1000, Some(2000), None) |
69 | 114 | listener.onBatchCompleted(StreamingListenerBatchCompleted(batchInfoCompleted)) |
@@ -116,4 +161,67 @@ class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { |
116 | 161 | listener.retainedCompletedBatches.size should be (limit) |
117 | 162 | listener.numTotalCompletedBatches should be(limit + 10) |
118 | 163 | } |
| 164 | + |
| 165 | + test("disorder onJobStart and onBatchXXX") { |
| 166 | + val ssc = setupStreams(input, operation) |
| 167 | + val limit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100) |
| 168 | + val listener = new StreamingJobProgressListener(ssc) |
| 169 | + |
| 170 | + // fulfill completedBatchInfos |
| 171 | + for(i <- 0 until limit) { |
| 172 | + val batchInfoCompleted = |
| 173 | + BatchInfo(Time(1000 + i * 100), Map.empty, 1000 + i * 100, Some(2000 + i * 100), None) |
| 174 | + val properties = new Properties() |
| 175 | + properties.setProperty(JobScheduler.BATCH_TIME_PROPERTY_KEY, (1000 + i * 100).toString) |
| 176 | + properties.setProperty(JobScheduler.OUTPUT_OP_ID_PROPERTY_KEY, "0") |
| 177 | + val jobStart = SparkListenerJobStart(jobId = 1, |
| 178 | + 0L, // unused |
| 179 | + Nil, // unused |
| 180 | + properties) |
| 181 | + listener.onBatchCompleted(StreamingListenerBatchCompleted(batchInfoCompleted)) |
| 182 | + listener.onJobStart(jobStart) |
| 183 | + } |
| 184 | + |
| 185 | + // onJobStart happens before onBatchSubmitted |
| 186 | + val properties = new Properties() |
| 187 | + properties.setProperty(JobScheduler.BATCH_TIME_PROPERTY_KEY, (1000 + limit * 100).toString) |
| 188 | + properties.setProperty(JobScheduler.OUTPUT_OP_ID_PROPERTY_KEY, "0") |
| 189 | + val jobStart = SparkListenerJobStart(jobId = 0, |
| 190 | + 0L, // unused |
| 191 | + Nil, // unused |
| 192 | + properties) |
| 193 | + listener.onJobStart(jobStart) |
| 194 | + |
| 195 | + val batchInfoSubmitted = |
| 196 | + BatchInfo(Time(1000 + limit * 100), Map.empty, (1000 + limit * 100), None, None) |
| 197 | + listener.onBatchSubmitted(StreamingListenerBatchSubmitted(batchInfoSubmitted)) |
| 198 | + |
| 199 | + // We still can see the info retrieved from onJobStart |
| 200 | + listener.getBatchUIData(Time(1000 + limit * 100)) should be |
| 201 | + Some(BatchUIData(batchInfoSubmitted, Seq((0, Seq(0))))) |
| 202 | + |
| 203 | + |
| 204 | + // A lot of "onBatchCompleted"s happen before "onJobStart" |
| 205 | + for(i <- limit + 1 to limit * 2) { |
| 206 | + val batchInfoCompleted = |
| 207 | + BatchInfo(Time(1000 + i * 100), Map.empty, 1000 + i * 100, Some(2000 + i * 100), None) |
| 208 | + listener.onBatchCompleted(StreamingListenerBatchCompleted(batchInfoCompleted)) |
| 209 | + } |
| 210 | + |
| 211 | + for(i <- limit + 1 to limit * 2) { |
| 212 | + val properties = new Properties() |
| 213 | + properties.setProperty(JobScheduler.BATCH_TIME_PROPERTY_KEY, (1000 + i * 100).toString) |
| 214 | + properties.setProperty(JobScheduler.OUTPUT_OP_ID_PROPERTY_KEY, "0") |
| 215 | + val jobStart = SparkListenerJobStart(jobId = 1, |
| 216 | + 0L, // unused |
| 217 | + Nil, // unused |
| 218 | + properties) |
| 219 | + listener.onJobStart(jobStart) |
| 220 | + } |
| 221 | + |
| 222 | + // We should not leak memory |
| 223 | + listener.batchTimeToOutputOpIdToSparkJobIds.size() should be <= |
| 224 | + (listener.waitingBatches.size + listener.runningBatches.size + |
| 225 | + listener.retainedCompletedBatches.size + 10) |
| 226 | + } |
119 | 227 | } |
0 commit comments