aboutsummaryrefslogtreecommitdiff
path: root/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
blob: 9305b6d9738e14e3d4f8bcdd8d17d722884c3172 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.spark.util

import java.util.Properties

import scala.collection.Map

import org.json4s.jackson.JsonMethods._
import org.scalatest.FunSuite

import org.apache.spark._
import org.apache.spark.executor._
import org.apache.spark.scheduler._
import org.apache.spark.storage._

class JsonProtocolSuite extends FunSuite {

  test("SparkListenerEvent") {
    val stageSubmitted =
      SparkListenerStageSubmitted(makeStageInfo(100, 200, 300, 400L, 500L), properties)
    val stageCompleted = SparkListenerStageCompleted(makeStageInfo(101, 201, 301, 401L, 501L))
    val taskStart = SparkListenerTaskStart(111, makeTaskInfo(222L, 333, 1, 444L, false))
    val taskGettingResult =
      SparkListenerTaskGettingResult(makeTaskInfo(1000L, 2000, 5, 3000L, true))
    val taskEnd = SparkListenerTaskEnd(1, "ShuffleMapTask", Success,
      makeTaskInfo(123L, 234, 67, 345L, false),
      makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = false))
    val taskEndWithHadoopInput = SparkListenerTaskEnd(1, "ShuffleMapTask", Success,
      makeTaskInfo(123L, 234, 67, 345L, false),
      makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true))
    val jobStart = SparkListenerJobStart(10, Seq[Int](1, 2, 3, 4), properties)
    val jobEnd = SparkListenerJobEnd(20, JobSucceeded)
    val environmentUpdate = SparkListenerEnvironmentUpdate(Map[String, Seq[(String, String)]](
      "JVM Information" -> Seq(("GC speed", "9999 objects/s"), ("Java home", "Land of coffee")),
      "Spark Properties" -> Seq(("Job throughput", "80000 jobs/s, regardless of job type")),
      "System Properties" -> Seq(("Username", "guest"), ("Password", "guest")),
      "Classpath Entries" -> Seq(("Super library", "/tmp/super_library"))
    ))
    val blockManagerAdded = SparkListenerBlockManagerAdded(
      BlockManagerId("Stars", "In your multitude...", 300, 400), 500)
    val blockManagerRemoved = SparkListenerBlockManagerRemoved(
      BlockManagerId("Scarce", "to be counted...", 100, 200))
    val unpersistRdd = SparkListenerUnpersistRDD(12345)
    val applicationStart = SparkListenerApplicationStart("The winner of all", 42L, "Garfield")
    val applicationEnd = SparkListenerApplicationEnd(42L)

    testEvent(stageSubmitted, stageSubmittedJsonString)
    testEvent(stageCompleted, stageCompletedJsonString)
    testEvent(taskStart, taskStartJsonString)
    testEvent(taskGettingResult, taskGettingResultJsonString)
    testEvent(taskEnd, taskEndJsonString)
    testEvent(taskEndWithHadoopInput, taskEndWithHadoopInputJsonString)
    testEvent(jobStart, jobStartJsonString)
    testEvent(jobEnd, jobEndJsonString)
    testEvent(environmentUpdate, environmentUpdateJsonString)
    testEvent(blockManagerAdded, blockManagerAddedJsonString)
    testEvent(blockManagerRemoved, blockManagerRemovedJsonString)
    testEvent(unpersistRdd, unpersistRDDJsonString)
    testEvent(applicationStart, applicationStartJsonString)
    testEvent(applicationEnd, applicationEndJsonString)
  }

  test("Dependent Classes") {
    testRDDInfo(makeRddInfo(2, 3, 4, 5L, 6L))
    testStageInfo(makeStageInfo(10, 20, 30, 40L, 50L))
    testTaskInfo(makeTaskInfo(999L, 888, 55, 777L, false))
    testTaskMetrics(makeTaskMetrics(33333L, 44444L, 55555L, 66666L, 7, 8, hasHadoopInput = false))
    testBlockManagerId(BlockManagerId("Hong", "Kong", 500, 1000))

    // StorageLevel
    testStorageLevel(StorageLevel.NONE)
    testStorageLevel(StorageLevel.DISK_ONLY)
    testStorageLevel(StorageLevel.DISK_ONLY_2)
    testStorageLevel(StorageLevel.MEMORY_ONLY)
    testStorageLevel(StorageLevel.MEMORY_ONLY_2)
    testStorageLevel(StorageLevel.MEMORY_ONLY_SER)
    testStorageLevel(StorageLevel.MEMORY_ONLY_SER_2)
    testStorageLevel(StorageLevel.MEMORY_AND_DISK)
    testStorageLevel(StorageLevel.MEMORY_AND_DISK_2)
    testStorageLevel(StorageLevel.MEMORY_AND_DISK_SER)
    testStorageLevel(StorageLevel.MEMORY_AND_DISK_SER_2)

    // JobResult
    val exception = new Exception("Out of Memory! Please restock film.")
    exception.setStackTrace(stackTrace)
    val jobFailed = JobFailed(exception)
    testJobResult(JobSucceeded)
    testJobResult(jobFailed)

    // TaskEndReason
    val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15, 16), 17, 18, 19)
    val exceptionFailure = ExceptionFailure("To be", "or not to be", stackTrace, None)
    testTaskEndReason(Success)
    testTaskEndReason(Resubmitted)
    testTaskEndReason(fetchFailed)
    testTaskEndReason(exceptionFailure)
    testTaskEndReason(TaskResultLost)
    testTaskEndReason(TaskKilled)
    testTaskEndReason(ExecutorLostFailure)
    testTaskEndReason(UnknownReason)

    // BlockId
    testBlockId(RDDBlockId(1, 2))
    testBlockId(ShuffleBlockId(1, 2, 3))
    testBlockId(BroadcastBlockId(1L, "insert_words_of_wisdom_here"))
    testBlockId(TaskResultBlockId(1L))
    testBlockId(StreamBlockId(1, 2L))
  }

  test("StageInfo.details backward compatibility") {
    // StageInfo.details was added after 1.0.0.
    val info = makeStageInfo(1, 2, 3, 4L, 5L)
    assert(info.details.nonEmpty)
    val newJson = JsonProtocol.stageInfoToJson(info)
    val oldJson = newJson.removeField { case (field, _) => field == "Details" }
    val newInfo = JsonProtocol.stageInfoFromJson(oldJson)
    assert(info.name === newInfo.name)
    assert("" === newInfo.details)
  }

  test("InputMetrics backward compatibility") {
    // InputMetrics were added after 1.0.1.
    val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6, hasHadoopInput = true)
    assert(metrics.inputMetrics.nonEmpty)
    val newJson = JsonProtocol.taskMetricsToJson(metrics)
    val oldJson = newJson.removeField { case (field, _) => field == "Input Metrics" }
    val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson)
    assert(newMetrics.inputMetrics.isEmpty)
  }


  /** -------------------------- *
   | Helper test running methods |
   * --------------------------- */

  private def testEvent(event: SparkListenerEvent, jsonString: String) {
    val actualJsonString = compact(render(JsonProtocol.sparkEventToJson(event)))
    val newEvent = JsonProtocol.sparkEventFromJson(parse(actualJsonString))
    assertJsonStringEquals(jsonString, actualJsonString)
    assertEquals(event, newEvent)
  }

  private def testRDDInfo(info: RDDInfo) {
    val newInfo = JsonProtocol.rddInfoFromJson(JsonProtocol.rddInfoToJson(info))
    assertEquals(info, newInfo)
  }

  private def testStageInfo(info: StageInfo) {
    val newInfo = JsonProtocol.stageInfoFromJson(JsonProtocol.stageInfoToJson(info))
    assertEquals(info, newInfo)
  }

  private def testStorageLevel(level: StorageLevel) {
    val newLevel = JsonProtocol.storageLevelFromJson(JsonProtocol.storageLevelToJson(level))
    assertEquals(level, newLevel)
  }

  private def testTaskMetrics(metrics: TaskMetrics) {
    val newMetrics = JsonProtocol.taskMetricsFromJson(JsonProtocol.taskMetricsToJson(metrics))
    assertEquals(metrics, newMetrics)
  }

  private def testBlockManagerId(id: BlockManagerId) {
    val newId = JsonProtocol.blockManagerIdFromJson(JsonProtocol.blockManagerIdToJson(id))
    assertEquals(id, newId)
  }

  private def testTaskInfo(info: TaskInfo) {
    val newInfo = JsonProtocol.taskInfoFromJson(JsonProtocol.taskInfoToJson(info))
    assertEquals(info, newInfo)
  }

  private def testJobResult(result: JobResult) {
    val newResult = JsonProtocol.jobResultFromJson(JsonProtocol.jobResultToJson(result))
    assertEquals(result, newResult)
  }

  private def testTaskEndReason(reason: TaskEndReason) {
    val newReason = JsonProtocol.taskEndReasonFromJson(JsonProtocol.taskEndReasonToJson(reason))
    assertEquals(reason, newReason)
  }

  private def testBlockId(blockId: BlockId) {
    val newBlockId = BlockId(blockId.toString)
    assert(blockId === newBlockId)
  }


  /** -------------------------------- *
   | Util methods for comparing events |
   * --------------------------------- */

  private def assertEquals(event1: SparkListenerEvent, event2: SparkListenerEvent) {
    (event1, event2) match {
      case (e1: SparkListenerStageSubmitted, e2: SparkListenerStageSubmitted) =>
        assert(e1.properties === e2.properties)
        assertEquals(e1.stageInfo, e2.stageInfo)
      case (e1: SparkListenerStageCompleted, e2: SparkListenerStageCompleted) =>
        assertEquals(e1.stageInfo, e2.stageInfo)
      case (e1: SparkListenerTaskStart, e2: SparkListenerTaskStart) =>
        assert(e1.stageId === e2.stageId)
        assertEquals(e1.taskInfo, e2.taskInfo)
      case (e1: SparkListenerTaskGettingResult, e2: SparkListenerTaskGettingResult) =>
        assertEquals(e1.taskInfo, e2.taskInfo)
      case (e1: SparkListenerTaskEnd, e2: SparkListenerTaskEnd) =>
        assert(e1.stageId === e2.stageId)
        assert(e1.taskType === e2.taskType)
        assertEquals(e1.reason, e2.reason)
        assertEquals(e1.taskInfo, e2.taskInfo)
        assertEquals(e1.taskMetrics, e2.taskMetrics)
      case (e1: SparkListenerJobStart, e2: SparkListenerJobStart) =>
        assert(e1.jobId === e2.jobId)
        assert(e1.properties === e2.properties)
        assertSeqEquals(e1.stageIds, e2.stageIds, (i1: Int, i2: Int) => assert(i1 === i2))
      case (e1: SparkListenerJobEnd, e2: SparkListenerJobEnd) =>
        assert(e1.jobId === e2.jobId)
        assertEquals(e1.jobResult, e2.jobResult)
      case (e1: SparkListenerEnvironmentUpdate, e2: SparkListenerEnvironmentUpdate) =>
        assertEquals(e1.environmentDetails, e2.environmentDetails)
      case (e1: SparkListenerBlockManagerAdded, e2: SparkListenerBlockManagerAdded) =>
        assert(e1.maxMem === e2.maxMem)
        assertEquals(e1.blockManagerId, e2.blockManagerId)
      case (e1: SparkListenerBlockManagerRemoved, e2: SparkListenerBlockManagerRemoved) =>
        assertEquals(e1.blockManagerId, e2.blockManagerId)
      case (e1: SparkListenerUnpersistRDD, e2: SparkListenerUnpersistRDD) =>
        assert(e1.rddId == e2.rddId)
      case (e1: SparkListenerApplicationStart, e2: SparkListenerApplicationStart) =>
        assert(e1.appName == e2.appName)
        assert(e1.time == e2.time)
        assert(e1.sparkUser == e2.sparkUser)
      case (e1: SparkListenerApplicationEnd, e2: SparkListenerApplicationEnd) =>
        assert(e1.time == e2.time)
      case (SparkListenerShutdown, SparkListenerShutdown) =>
      case _ => fail("Events don't match in types!")
    }
  }

  private def assertEquals(info1: StageInfo, info2: StageInfo) {
    assert(info1.stageId === info2.stageId)
    assert(info1.name === info2.name)
    assert(info1.numTasks === info2.numTasks)
    assert(info1.submissionTime === info2.submissionTime)
    assert(info1.completionTime === info2.completionTime)
    assert(info1.rddInfos.size === info2.rddInfos.size)
    (0 until info1.rddInfos.size).foreach { i =>
      assertEquals(info1.rddInfos(i), info2.rddInfos(i))
    }
    assert(info1.details === info2.details)
  }

  private def assertEquals(info1: RDDInfo, info2: RDDInfo) {
    assert(info1.id === info2.id)
    assert(info1.name === info2.name)
    assert(info1.numPartitions === info2.numPartitions)
    assert(info1.numCachedPartitions === info2.numCachedPartitions)
    assert(info1.memSize === info2.memSize)
    assert(info1.diskSize === info2.diskSize)
    assertEquals(info1.storageLevel, info2.storageLevel)
  }

  private def assertEquals(level1: StorageLevel, level2: StorageLevel) {
    assert(level1.useDisk === level2.useDisk)
    assert(level1.useMemory === level2.useMemory)
    assert(level1.deserialized === level2.deserialized)
    assert(level1.replication === level2.replication)
  }

  private def assertEquals(info1: TaskInfo, info2: TaskInfo) {
    assert(info1.taskId === info2.taskId)
    assert(info1.index === info2.index)
    assert(info1.attempt === info2.attempt)
    assert(info1.launchTime === info2.launchTime)
    assert(info1.executorId === info2.executorId)
    assert(info1.host === info2.host)
    assert(info1.taskLocality === info2.taskLocality)
    assert(info1.speculative === info2.speculative)
    assert(info1.gettingResultTime === info2.gettingResultTime)
    assert(info1.finishTime === info2.finishTime)
    assert(info1.failed === info2.failed)
  }

  private def assertEquals(metrics1: TaskMetrics, metrics2: TaskMetrics) {
    assert(metrics1.hostname === metrics2.hostname)
    assert(metrics1.executorDeserializeTime === metrics2.executorDeserializeTime)
    assert(metrics1.resultSize === metrics2.resultSize)
    assert(metrics1.jvmGCTime === metrics2.jvmGCTime)
    assert(metrics1.resultSerializationTime === metrics2.resultSerializationTime)
    assert(metrics1.memoryBytesSpilled === metrics2.memoryBytesSpilled)
    assert(metrics1.diskBytesSpilled === metrics2.diskBytesSpilled)
    assertOptionEquals(
      metrics1.shuffleReadMetrics, metrics2.shuffleReadMetrics, assertShuffleReadEquals)
    assertOptionEquals(
      metrics1.shuffleWriteMetrics, metrics2.shuffleWriteMetrics, assertShuffleWriteEquals)
    assertOptionEquals(
      metrics1.inputMetrics, metrics2.inputMetrics, assertInputMetricsEquals)
    assertOptionEquals(metrics1.updatedBlocks, metrics2.updatedBlocks, assertBlocksEquals)
  }

  private def assertEquals(metrics1: ShuffleReadMetrics, metrics2: ShuffleReadMetrics) {
    assert(metrics1.shuffleFinishTime === metrics2.shuffleFinishTime)
    assert(metrics1.remoteBlocksFetched === metrics2.remoteBlocksFetched)
    assert(metrics1.localBlocksFetched === metrics2.localBlocksFetched)
    assert(metrics1.fetchWaitTime === metrics2.fetchWaitTime)
    assert(metrics1.remoteBytesRead === metrics2.remoteBytesRead)
  }

  private def assertEquals(metrics1: ShuffleWriteMetrics, metrics2: ShuffleWriteMetrics) {
    assert(metrics1.shuffleBytesWritten === metrics2.shuffleBytesWritten)
    assert(metrics1.shuffleWriteTime === metrics2.shuffleWriteTime)
  }

  private def assertEquals(metrics1: InputMetrics, metrics2: InputMetrics) {
    assert(metrics1.readMethod === metrics2.readMethod)
    assert(metrics1.bytesRead === metrics2.bytesRead)
  }

  private def assertEquals(bm1: BlockManagerId, bm2: BlockManagerId) {
    assert(bm1.executorId === bm2.executorId)
    assert(bm1.host === bm2.host)
    assert(bm1.port === bm2.port)
    assert(bm1.nettyPort === bm2.nettyPort)
  }

  private def assertEquals(result1: JobResult, result2: JobResult) {
    (result1, result2) match {
      case (JobSucceeded, JobSucceeded) =>
      case (r1: JobFailed, r2: JobFailed) =>
        assertEquals(r1.exception, r2.exception)
      case _ => fail("Job results don't match in types!")
    }
  }

  private def assertEquals(reason1: TaskEndReason, reason2: TaskEndReason) {
    (reason1, reason2) match {
      case (Success, Success) =>
      case (Resubmitted, Resubmitted) =>
      case (r1: FetchFailed, r2: FetchFailed) =>
        assert(r1.shuffleId === r2.shuffleId)
        assert(r1.mapId === r2.mapId)
        assert(r1.reduceId === r2.reduceId)
        assertEquals(r1.bmAddress, r2.bmAddress)
      case (r1: ExceptionFailure, r2: ExceptionFailure) =>
        assert(r1.className === r2.className)
        assert(r1.description === r2.description)
        assertSeqEquals(r1.stackTrace, r2.stackTrace, assertStackTraceElementEquals)
        assertOptionEquals(r1.metrics, r2.metrics, assertTaskMetricsEquals)
      case (TaskResultLost, TaskResultLost) =>
      case (TaskKilled, TaskKilled) =>
      case (ExecutorLostFailure, ExecutorLostFailure) =>
      case (UnknownReason, UnknownReason) =>
      case _ => fail("Task end reasons don't match in types!")
    }
  }

  private def assertEquals(
      details1: Map[String, Seq[(String, String)]],
      details2: Map[String, Seq[(String, String)]]) {
    details1.zip(details2).foreach {
      case ((key1, values1: Seq[(String, String)]), (key2, values2: Seq[(String, String)])) =>
        assert(key1 === key2)
        values1.zip(values2).foreach { case (v1, v2) => assert(v1 === v2) }
    }
  }

  private def assertEquals(exception1: Exception, exception2: Exception) {
    assert(exception1.getMessage === exception2.getMessage)
    assertSeqEquals(
      exception1.getStackTrace,
      exception2.getStackTrace,
      assertStackTraceElementEquals)
  }

  private def assertJsonStringEquals(json1: String, json2: String) {
    val formatJsonString = (json: String) => json.replaceAll("[\\s|]", "")
    assert(formatJsonString(json1) === formatJsonString(json2))
  }

  private def assertSeqEquals[T](seq1: Seq[T], seq2: Seq[T], assertEquals: (T, T) => Unit) {
    assert(seq1.length === seq2.length)
    seq1.zip(seq2).foreach { case (t1, t2) =>
      assertEquals(t1, t2)
    }
  }

  private def assertOptionEquals[T](
      opt1: Option[T],
      opt2: Option[T],
      assertEquals: (T, T) => Unit) {
    if (opt1.isDefined) {
      assert(opt2.isDefined)
      assertEquals(opt1.get, opt2.get)
    } else {
      assert(!opt2.isDefined)
    }
  }

  /**
   * Use different names for methods we pass in to assertSeqEquals or assertOptionEquals
   */

  private def assertShuffleReadEquals(r1: ShuffleReadMetrics, r2: ShuffleReadMetrics) {
    assertEquals(r1, r2)
  }

  private def assertShuffleWriteEquals(w1: ShuffleWriteMetrics, w2: ShuffleWriteMetrics) {
    assertEquals(w1, w2)
  }

  private def assertInputMetricsEquals(i1: InputMetrics, i2: InputMetrics) {
    assertEquals(i1, i2)
  }

  private def assertTaskMetricsEquals(t1: TaskMetrics, t2: TaskMetrics) {
    assertEquals(t1, t2)
  }

  private def assertBlocksEquals(
      blocks1: Seq[(BlockId, BlockStatus)],
      blocks2: Seq[(BlockId, BlockStatus)]) = {
    assertSeqEquals(blocks1, blocks2, assertBlockEquals)
  }

  private def assertBlockEquals(b1: (BlockId, BlockStatus), b2: (BlockId, BlockStatus)) {
    assert(b1 === b2)
  }

  private def assertStackTraceElementEquals(ste1: StackTraceElement, ste2: StackTraceElement) {
    assert(ste1 === ste2)
  }


  /** ----------------------------------- *
   | Util methods for constructing events |
   * ------------------------------------ */

  private val properties = {
    val p = new Properties
    p.setProperty("Ukraine", "Kiev")
    p.setProperty("Russia", "Moscow")
    p.setProperty("France", "Paris")
    p.setProperty("Germany", "Berlin")
    p
  }

  private val stackTrace = {
    Array[StackTraceElement](
      new StackTraceElement("Apollo", "Venus", "Mercury", 42),
      new StackTraceElement("Afollo", "Vemus", "Mercurry", 420),
      new StackTraceElement("Ayollo", "Vesus", "Blackberry", 4200)
    )
  }

  private def makeRddInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = {
    val r = new RDDInfo(a, "mayor", b, StorageLevel.MEMORY_AND_DISK)
    r.numCachedPartitions = c
    r.memSize = d
    r.diskSize = e
    r
  }

  private def makeStageInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = {
    val rddInfos = (0 until a % 5).map { i => makeRddInfo(a + i, b + i, c + i, d + i, e + i) }
    new StageInfo(a, "greetings", b, rddInfos, "details")
  }

  private def makeTaskInfo(a: Long, b: Int, c: Int, d: Long, speculative: Boolean) = {
    new TaskInfo(a, b, c, d, "executor", "your kind sir", TaskLocality.NODE_LOCAL, speculative)
  }

  /**
   * Creates a TaskMetrics object describing a task that read data from Hadoop (if hasHadoopInput is
   * set to true) or read data from a shuffle otherwise.
   */
  private def makeTaskMetrics(
      a: Long,
      b: Long,
      c: Long,
      d: Long,
      e: Int,
      f: Int,
      hasHadoopInput: Boolean) = {
    val t = new TaskMetrics
    val sw = new ShuffleWriteMetrics
    t.hostname = "localhost"
    t.executorDeserializeTime = a
    t.executorRunTime = b
    t.resultSize = c
    t.jvmGCTime = d
    t.resultSerializationTime = a + b
    t.memoryBytesSpilled = a + c

    if (hasHadoopInput) {
      val inputMetrics = new InputMetrics(DataReadMethod.Hadoop)
      inputMetrics.bytesRead = d + e + f
      t.inputMetrics = Some(inputMetrics)
    } else {
      val sr = new ShuffleReadMetrics
      sr.shuffleFinishTime = b + c
      sr.remoteBytesRead = b + d
      sr.localBlocksFetched = e
      sr.fetchWaitTime = a + d
      sr.remoteBlocksFetched = f
      t.updateShuffleReadMetrics(sr)
    }
    sw.shuffleBytesWritten = a + b + c
    sw.shuffleWriteTime = b + c + d
    t.shuffleWriteMetrics = Some(sw)
    // Make at most 6 blocks
    t.updatedBlocks = Some((1 to (e % 5 + 1)).map { i =>
      (RDDBlockId(e % i, f % i), BlockStatus(StorageLevel.MEMORY_AND_DISK_SER_2, a % i, b % i, c%i))
    }.toSeq)
    t
  }


  /** --------------------------------------- *
   | JSON string representation of each event |
   * ---------------------------------------- */

  private val stageSubmittedJsonString =
    """
      {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":100,"Stage Name":
      "greetings","Number of Tasks":200,"RDD Info":[],"Details":"details"},"Properties":
      {"France":"Paris","Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}}
    """

  private val stageCompletedJsonString =
    """
      {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":101,"Stage Name":
      "greetings","Number of Tasks":201,"RDD Info":[{"RDD ID":101,"Name":"mayor","Storage
      Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,
      "Replication":1},"Number of Partitions":201,"Number of Cached Partitions":301,
      "Memory Size":401,"Tachyon Size":0,"Disk Size":501}],"Details":"details"}}
    """

  private val taskStartJsonString =
    """
      |{"Event":"SparkListenerTaskStart","Stage ID":111,"Task Info":{"Task ID":222,
      |"Index":333,"Attempt":1,"Launch Time":444,"Executor ID":"executor","Host":"your kind sir",
      |"Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,
      |"Failed":false}}
    """.stripMargin

  private val taskGettingResultJsonString =
    """
      |{"Event":"SparkListenerTaskGettingResult","Task Info":
      |  {"Task ID":1000,"Index":2000,"Attempt":5,"Launch Time":3000,"Executor ID":"executor",
      |   "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":true,"Getting Result Time":0,
      |   "Finish Time":0,"Failed":false
      |  }
      |}
    """.stripMargin

  private val taskEndJsonString =
    """
      |{"Event":"SparkListenerTaskEnd","Stage ID":1,"Task Type":"ShuffleMapTask",
      |"Task End Reason":{"Reason":"Success"},
      |"Task Info":{
      |  "Task ID":123,"Index":234,"Attempt":67,"Launch Time":345,"Executor ID":"executor",
      |  "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":false,
      |  "Getting Result Time":0,"Finish Time":0,"Failed":false
      |},
      |"Task Metrics":{
      |  "Host Name":"localhost","Executor Deserialize Time":300,"Executor Run Time":400,
      |  "Result Size":500,"JVM GC Time":600,"Result Serialization Time":700,
      |  "Memory Bytes Spilled":800,"Disk Bytes Spilled":0,
      |  "Shuffle Read Metrics":{
      |    "Shuffle Finish Time":900,
      |    "Remote Blocks Fetched":800,
      |    "Local Blocks Fetched":700,
      |    "Fetch Wait Time":900,
      |    "Remote Bytes Read":1000
      |  },
      |  "Shuffle Write Metrics":{
      |    "Shuffle Bytes Written":1200,
      |    "Shuffle Write Time":1500
      |  },
      |  "Updated Blocks":[
      |    {"Block ID":"rdd_0_0",
      |      "Status":{
      |        "Storage Level":{
      |          "Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":false,
      |          "Replication":2
      |        },
      |        "Memory Size":0,"Tachyon Size":0,"Disk Size":0
      |      }
      |    }
      |    ]
      |  }
      |}
    """.stripMargin

  private val taskEndWithHadoopInputJsonString =
    """
      |{"Event":"SparkListenerTaskEnd","Stage ID":1,"Task Type":"ShuffleMapTask",
      |"Task End Reason":{"Reason":"Success"},
      |"Task Info":{
      |  "Task ID":123,"Index":234,"Attempt":67,"Launch Time":345,"Executor ID":"executor",
      |  "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":false,
      |  "Getting Result Time":0,"Finish Time":0,"Failed":false
      |},
      |"Task Metrics":{
      |  "Host Name":"localhost","Executor Deserialize Time":300,"Executor Run Time":400,
      |  "Result Size":500,"JVM GC Time":600,"Result Serialization Time":700,
      |  "Memory Bytes Spilled":800,"Disk Bytes Spilled":0,
      |  "Shuffle Write Metrics":{"Shuffle Bytes Written":1200,"Shuffle Write Time":1500},
      |  "Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":2100},
      |  "Updated Blocks":[
      |    {"Block ID":"rdd_0_0",
      |      "Status":{
      |        "Storage Level":{
      |          "Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":false,
      |          "Replication":2
      |        },
      |        "Memory Size":0,"Tachyon Size":0,"Disk Size":0
      |      }
      |    }
      |  ]}
      |}
    """

  private val jobStartJsonString =
    """
      {"Event":"SparkListenerJobStart","Job ID":10,"Stage IDs":[1,2,3,4],"Properties":
      {"France":"Paris","Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}}
    """

  private val jobEndJsonString =
    """
      {"Event":"SparkListenerJobEnd","Job ID":20,"Job Result":{"Result":"JobSucceeded"}}
    """

  private val environmentUpdateJsonString =
    """
      {"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"GC speed":"9999 objects/s",
      "Java home":"Land of coffee"},"Spark Properties":{"Job throughput":"80000 jobs/s,
      regardless of job type"},"System Properties":{"Username":"guest","Password":"guest"},
      "Classpath Entries":{"Super library":"/tmp/super_library"}}
    """

  private val blockManagerAddedJsonString =
    """
      {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"Stars",
      "Host":"In your multitude...","Port":300,"Netty Port":400},"Maximum Memory":500}
    """

  private val blockManagerRemovedJsonString =
    """
      {"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"Scarce",
      "Host":"to be counted...","Port":100,"Netty Port":200}}
    """

  private val unpersistRDDJsonString =
    """
      {"Event":"SparkListenerUnpersistRDD","RDD ID":12345}
    """

  private val applicationStartJsonString =
    """
      {"Event":"SparkListenerApplicationStart","App Name":"The winner of all","Timestamp":42,
      "User":"Garfield"}
    """

  private val applicationEndJsonString =
    """
      {"Event":"SparkListenerApplicationEnd","Timestamp":42}
    """
}