[SPARK-15878][CORE][TEST] fix cleanup in EventLoggingListenerSuite and ReplayListenerSuite

## What changes were proposed in this pull request?

These tests weren't properly using `LocalSparkContext` so weren't cleaning up correctly when tests failed.

## How was this patch tested?

Jenkins.

Author: Imran Rashid <irashid@cloudera.com>

Closes #13602 from squito/SPARK-15878_cleanup_replaylistener.
This commit is contained in:
Imran Rashid 2016-06-12 12:54:57 +01:00 committed by Sean Owen
parent 9e204c62c6
commit 8cc22b0085
2 changed files with 4 additions and 4 deletions

View file

@ -181,7 +181,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
// into SPARK-6688. // into SPARK-6688.
val conf = getLoggingConf(testDirPath, compressionCodec) val conf = getLoggingConf(testDirPath, compressionCodec)
.set("spark.hadoop.fs.defaultFS", "unsupported://example.com") .set("spark.hadoop.fs.defaultFS", "unsupported://example.com")
val sc = new SparkContext("local-cluster[2,2,1024]", "test", conf) sc = new SparkContext("local-cluster[2,2,1024]", "test", conf)
assert(sc.eventLogger.isDefined) assert(sc.eventLogger.isDefined)
val eventLogger = sc.eventLogger.get val eventLogger = sc.eventLogger.get
val eventLogPath = eventLogger.logPath val eventLogPath = eventLogger.logPath

View file

@ -23,7 +23,7 @@ import java.net.URI
import org.json4s.jackson.JsonMethods._ import org.json4s.jackson.JsonMethods._
import org.scalatest.BeforeAndAfter import org.scalatest.BeforeAndAfter
import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite}
import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.io.CompressionCodec import org.apache.spark.io.CompressionCodec
import org.apache.spark.util.{JsonProtocol, JsonProtocolSuite, Utils} import org.apache.spark.util.{JsonProtocol, JsonProtocolSuite, Utils}
@ -31,7 +31,7 @@ import org.apache.spark.util.{JsonProtocol, JsonProtocolSuite, Utils}
/** /**
* Test whether ReplayListenerBus replays events from logs correctly. * Test whether ReplayListenerBus replays events from logs correctly.
*/ */
class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter { class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSparkContext {
private val fileSystem = Utils.getHadoopFileSystem("/", private val fileSystem = Utils.getHadoopFileSystem("/",
SparkHadoopUtil.get.newConfiguration(new SparkConf())) SparkHadoopUtil.get.newConfiguration(new SparkConf()))
private var testDir: File = _ private var testDir: File = _
@ -101,7 +101,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter {
fileSystem.mkdirs(logDirPath) fileSystem.mkdirs(logDirPath)
val conf = EventLoggingListenerSuite.getLoggingConf(logDirPath, codecName) val conf = EventLoggingListenerSuite.getLoggingConf(logDirPath, codecName)
val sc = new SparkContext("local-cluster[2,1,1024]", "Test replay", conf) sc = new SparkContext("local-cluster[2,1,1024]", "Test replay", conf)
// Run a few jobs // Run a few jobs
sc.parallelize(1 to 100, 1).count() sc.parallelize(1 to 100, 1).count()