diff options
author | Mingfei <mingfei.shi@intel.com> | 2013-06-13 14:36:07 +0800 |
---|---|---|
committer | Mingfei <mingfei.shi@intel.com> | 2013-06-13 14:36:07 +0800 |
commit | 967a6a699da7da007f51e59d085a357da5ec14da (patch) | |
tree | 3979c761746d7b60130495f374ebddbcb7c020e9 /core/src/test/scala | |
parent | ade822011d44bd43e9ac78c1d29ec924a1f6e8e7 (diff) | |
download | spark-967a6a699da7da007f51e59d085a357da5ec14da.tar.gz spark-967a6a699da7da007f51e59d085a357da5ec14da.tar.bz2 spark-967a6a699da7da007f51e59d085a357da5ec14da.zip |
modify sparklister function interface according to comments
Diffstat (limited to 'core/src/test/scala')
-rw-r--r-- | core/src/test/scala/spark/scheduler/JobLoggerSuite.scala | 10 |
1 files changed, 5 insertions, 5 deletions
diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala index 34fd8b995e..a654bf3ffd 100644 --- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala @@ -40,7 +40,7 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val shuffleMapStage = new Stage(1, parentRdd, Some(shuffleDep), Nil, jobID) val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID) - joblogger.onStageSubmitted(rootStage) + joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStage, 4)) joblogger.getEventQueue.size should be (1) joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName) parentRdd.setName("MyRDD") @@ -86,11 +86,11 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers var onJobStartCount = 0 var onStageCompletedCount = 0 var onStageSubmittedCount = 0 - override def onTaskEnd(event: CompletionEvent) = onTaskEndCount += 1 - override def onJobEnd(job: ActiveJob, event: SparkListenerEvents) = onJobEndCount += 1 - override def onJobStart(job: ActiveJob, properties: Properties) = onJobStartCount += 1 + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = onTaskEndCount += 1 + override def onJobEnd(jobEnd: SparkListenerEvents) = onJobEndCount += 1 + override def onJobStart(jobStart: SparkListenerJobStart) = onJobStartCount += 1 override def onStageCompleted(stageCompleted: StageCompleted) = onStageCompletedCount += 1 - override def onStageSubmitted(stage: Stage, info: String = "") = onStageSubmittedCount += 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) } |