diff options
Diffstat (limited to 'streaming/src/test')
4 files changed, 8 insertions, 2 deletions
diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index f9bfb9b744..ce58cb12a4 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -17,7 +17,6 @@ package org.apache.spark.streaming; -import org.apache.spark.streaming.api.java.*; import scala.Tuple2; import org.junit.Assert; @@ -37,6 +36,8 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.*; import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.api.java.*; +import org.apache.spark.util.Utils; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; @@ -1606,6 +1607,7 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa Arrays.asList(8,7)); File tempDir = Files.createTempDir(); + tempDir.deleteOnExit(); ssc.checkpoint(tempDir.getAbsolutePath()); JavaDStream<String> stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1); @@ -1627,6 +1629,7 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa // will be re-processed after recovery List<List<Integer>> finalResult = JavaCheckpointTestUtils.runStreams(ssc, 2, 3); assertOrderInvariantEquals(expectedFinal, finalResult.subList(1, 3)); + Utils.deleteRecursively(tempDir); } diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index 063529a9cb..45d2ec676d 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=streaming/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 25739956cb..d20a7b728c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -232,6 +232,7 @@ class CheckpointSuite extends TestSuiteBase { test("recovery with file input stream") { // Set up the streaming context and input streams val testDir = Files.createTempDir() + testDir.deleteOnExit() var ssc = new StreamingContext(master, framework, Seconds(1)) ssc.checkpoint(checkpointDir) val fileStream = ssc.textFileStream(testDir.toString) @@ -326,6 +327,7 @@ class CheckpointSuite extends TestSuiteBase { ) // To ensure that all the inputs were received correctly assert(expectedOutput.last === output.last) + Utils.deleteRecursively(testDir) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 3fa254065c..cd0aa4d0dc 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -98,6 +98,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Set up the streaming context and input streams val testDir = Files.createTempDir() + testDir.deleteOnExit() val ssc = new StreamingContext(conf, batchDuration) val fileStream = ssc.textFileStream(testDir.toString) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] |