aboutsummaryrefslogtreecommitdiff
path: root/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala
blob: 4000c4d5209b6ff96410d17b90e17b366c75d0fb (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
package spark.scheduler

import java.util.Properties
import java.util.concurrent.LinkedBlockingQueue
import org.scalatest.FunSuite
import org.scalatest.matchers.ShouldMatchers
import scala.collection.mutable
import spark._
import spark.SparkContext._


class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers {

  test("inner method") {
    sc = new SparkContext("local", "joblogger")
    val joblogger = new JobLogger {
      def createLogWriterTest(jobID: Int) = createLogWriter(jobID)
      def closeLogWriterTest(jobID: Int) = closeLogWriter(jobID)
      def getRddNameTest(rdd: RDD[_]) = getRddName(rdd)
      def buildJobDepTest(jobID: Int, stage: Stage) = buildJobDep(jobID, stage) 
    }
    type MyRDD = RDD[(Int, Int)]
    def makeRdd(
        numPartitions: Int,
        dependencies: List[Dependency[_]]
      ): MyRDD = {
      val maxPartition = numPartitions - 1
      return new MyRDD(sc, dependencies) {
        override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] =
          throw new RuntimeException("should not be reached")
        override def getPartitions = (0 to maxPartition).map(i => new Partition {
          override def index = i
        }).toArray
      }
    }
    val jobID = 5
    val parentRdd = makeRdd(4, Nil)
    val shuffleDep = new ShuffleDependency(parentRdd, null)
    val rootRdd = makeRdd(4, List(shuffleDep))
    val shuffleMapStage = new Stage(1, parentRdd, Some(shuffleDep), Nil, jobID) 
    val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID)
    
    joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStage, 4))
    joblogger.getEventQueue.size should be (1)
    joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName)
    parentRdd.setName("MyRDD")
    joblogger.getRddNameTest(parentRdd) should be ("MyRDD")
    joblogger.createLogWriterTest(jobID)
    joblogger.getJobIDtoPrintWriter.size should be (1)
    joblogger.buildJobDepTest(jobID, rootStage)
    joblogger.getJobIDToStages.get(jobID).get.size should be (2)
    joblogger.getStageIDToJobID.get(0) should be (Some(jobID))
    joblogger.getStageIDToJobID.get(1) should be (Some(jobID))
    joblogger.closeLogWriterTest(jobID)
    joblogger.getStageIDToJobID.size should be (0)
    joblogger.getJobIDToStages.size should be (0)
    joblogger.getJobIDtoPrintWriter.size should be (0)
  }
  
  test("inner variables") {
    sc = new SparkContext("local[4]", "joblogger")
    val joblogger = new JobLogger {
      override protected def closeLogWriter(jobID: Int) = 
        getJobIDtoPrintWriter.get(jobID).foreach { fileWriter => 
          fileWriter.close()
        }
    }
    sc.addSparkListener(joblogger)
    val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) }
    rdd.reduceByKey(_+_).collect()
    
    joblogger.getLogDir should be ("/tmp/spark")
    joblogger.getJobIDtoPrintWriter.size should be (1)
    joblogger.getStageIDToJobID.size should be (2)
    joblogger.getStageIDToJobID.get(0) should be (Some(0))
    joblogger.getStageIDToJobID.get(1) should be (Some(0))
    joblogger.getJobIDToStages.size should be (1)
  }
  
  
  test("interface functions") {
    sc = new SparkContext("local[4]", "joblogger")
    val joblogger = new JobLogger {
      var onTaskEndCount = 0
      var onJobEndCount = 0 
      var onJobStartCount = 0
      var onStageCompletedCount = 0
      var onStageSubmittedCount = 0
      override def onTaskEnd(taskEnd: SparkListenerTaskEnd)  = onTaskEndCount += 1
      override def onJobEnd(jobEnd: SparkListenerJobEnd) = onJobEndCount += 1
      override def onJobStart(jobStart: SparkListenerJobStart) = onJobStartCount += 1
      override def onStageCompleted(stageCompleted: StageCompleted) = onStageCompletedCount += 1
      override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = onStageSubmittedCount += 1
    }
    sc.addSparkListener(joblogger)
    val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) }
    rdd.reduceByKey(_+_).collect()
    
    joblogger.onJobStartCount should be (1)
    joblogger.onJobEndCount should be (1)
    joblogger.onTaskEndCount should be (8)
    joblogger.onStageSubmittedCount should be (2)
    joblogger.onStageCompletedCount should be (2)
  }
}